You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2015/09/08 20:58:51 UTC
[01/15] flink git commit: [FLINK-2580] [runtime] Expose more methods
form Hadoop output streams and exposes wrapped input and output streams.
Repository: flink
Updated Branches:
refs/heads/master 18004343b -> 304139d48
[FLINK-2580] [runtime] Expose more methods form Hadoop output streams and exposes wrapped input and output streams.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/304139d4
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/304139d4
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/304139d4
Branch: refs/heads/master
Commit: 304139d48101d5c7eb9f0d5fa79cce37f235b0ab
Parents: b18e410
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Sep 1 13:16:46 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Sep 8 20:58:05 2015 +0200
----------------------------------------------------------------------
.../flink/core/fs/FSDataOutputStream.java | 7 +-
.../core/fs/local/LocalDataOutputStream.java | 11 ++
.../runtime/fs/hdfs/HadoopDataInputStream.java | 19 ++-
.../runtime/fs/hdfs/HadoopDataOutputStream.java | 132 ++++++++++++++++++-
.../flink/runtime/fs/hdfs/HadoopFileSystem.java | 22 +++-
.../flink/runtime/fs/s3/S3DataOutputStream.java | 54 +++-----
6 files changed, 191 insertions(+), 54 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/304139d4/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStream.java
index 831b29e..9974c9f 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStream.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStream.java
@@ -16,14 +16,17 @@
* limitations under the License.
*/
-
package org.apache.flink.core.fs;
+import java.io.IOException;
import java.io.OutputStream;
/**
* Interface for a data output stream to a file on a {@link FileSystem}.
- *
*/
public abstract class FSDataOutputStream extends OutputStream {
+
+ public abstract void flush() throws IOException;
+
+ public abstract void sync() throws IOException;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/304139d4/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java
index afef7c1..ea1ca34 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java
@@ -77,4 +77,15 @@ public class LocalDataOutputStream extends FSDataOutputStream {
public void close() throws IOException {
fos.close();
}
+
+
+ @Override
+ public void flush() throws IOException {
+ fos.flush();
+ }
+
+ @Override
+ public void sync() throws IOException {
+ fos.getFD().sync();
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/304139d4/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java
index 9a62885..6eeeb57 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java
@@ -16,7 +16,6 @@
* limitations under the License.
*/
-
package org.apache.flink.runtime.fs.hdfs;
import java.io.IOException;
@@ -26,11 +25,10 @@ import org.apache.flink.core.fs.FSDataInputStream;
/**
* Concrete implementation of the {@link FSDataInputStream} for the
* Hadoop Distributed File System.
- *
*/
public final class HadoopDataInputStream extends FSDataInputStream {
- private org.apache.hadoop.fs.FSDataInputStream fsDataInputStream = null;
+ private final org.apache.hadoop.fs.FSDataInputStream fsDataInputStream;
/**
* Creates a new data input stream from the given HDFS input stream
@@ -39,13 +37,15 @@ public final class HadoopDataInputStream extends FSDataInputStream {
* the HDFS input stream
*/
public HadoopDataInputStream(org.apache.hadoop.fs.FSDataInputStream fsDataInputStream) {
+ if (fsDataInputStream == null) {
+ throw new NullPointerException();
+ }
this.fsDataInputStream = fsDataInputStream;
}
@Override
public synchronized void seek(long desired) throws IOException {
-
fsDataInputStream.seek(desired);
}
@@ -68,17 +68,22 @@ public final class HadoopDataInputStream extends FSDataInputStream {
public int read(byte[] buffer, int offset, int length) throws IOException {
return fsDataInputStream.read(buffer, offset, length);
}
-
-
+
@Override
public int available() throws IOException {
return fsDataInputStream.available();
}
-
@Override
public long skip(long n) throws IOException {
return fsDataInputStream.skip(n);
}
+ /**
+ * Gets the wrapped Hadoop input stream.
+ * @return The wrapped Hadoop input stream.
+ */
+ public org.apache.hadoop.fs.FSDataInputStream getHadoopInputStream() {
+ return fsDataInputStream;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/304139d4/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataOutputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataOutputStream.java
index 3f6cb1c..b0ff4b3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataOutputStream.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataOutputStream.java
@@ -16,24 +16,27 @@
* limitations under the License.
*/
-
package org.apache.flink.runtime.fs.hdfs;
import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
import org.apache.flink.core.fs.FSDataOutputStream;
-public final class HadoopDataOutputStream extends FSDataOutputStream {
+public class HadoopDataOutputStream extends FSDataOutputStream {
- private org.apache.hadoop.fs.FSDataOutputStream fdos;
+ private final org.apache.hadoop.fs.FSDataOutputStream fdos;
public HadoopDataOutputStream(org.apache.hadoop.fs.FSDataOutputStream fdos) {
+ if (fdos == null) {
+ throw new NullPointerException();
+ }
this.fdos = fdos;
}
@Override
public void write(int b) throws IOException {
-
fdos.write(b);
}
@@ -47,4 +50,125 @@ public final class HadoopDataOutputStream extends FSDataOutputStream {
fdos.close();
}
+ @Override
+ public void flush() throws IOException {
+ if (HFLUSH_METHOD != null) {
+ try {
+ HFLUSH_METHOD.invoke(fdos);
+ }
+ catch (InvocationTargetException e) {
+ Throwable cause = e.getTargetException();
+ if (cause instanceof IOException) {
+ throw (IOException) cause;
+ }
+ else if (cause instanceof RuntimeException) {
+ throw (RuntimeException) cause;
+ }
+ else if (cause instanceof Error) {
+ throw (Error) cause;
+ }
+ else {
+ throw new IOException("Exception while invoking hflush()", cause);
+ }
+ }
+ catch (IllegalAccessException e) {
+ throw new IOException("Cannot invoke hflush()", e);
+ }
+ }
+ else if (HFLUSH_ERROR != null) {
+ if (HFLUSH_ERROR instanceof NoSuchMethodException) {
+ throw new UnsupportedOperationException("hflush() method is not available in this version of Hadoop.");
+ }
+ else {
+ throw new IOException("Cannot access hflush() method", HFLUSH_ERROR);
+ }
+ }
+ else {
+ throw new UnsupportedOperationException("hflush() is not available in this version of Hadoop.");
+ }
+ }
+
+ @Override
+ public void sync() throws IOException {
+ if (HSYNC_METHOD != null) {
+ try {
+ HSYNC_METHOD.invoke(fdos);
+ }
+ catch (InvocationTargetException e) {
+ Throwable cause = e.getTargetException();
+ if (cause instanceof IOException) {
+ throw (IOException) cause;
+ }
+ else if (cause instanceof RuntimeException) {
+ throw (RuntimeException) cause;
+ }
+ else if (cause instanceof Error) {
+ throw (Error) cause;
+ }
+ else {
+ throw new IOException("Exception while invoking hsync()", cause);
+ }
+ }
+ catch (IllegalAccessException e) {
+ throw new IOException("Cannot invoke hsync()", e);
+ }
+ }
+ else if (HSYNC_ERROR != null) {
+ if (HSYNC_ERROR instanceof NoSuchMethodException) {
+ throw new UnsupportedOperationException("hsync() method is not available in this version of Hadoop.");
+ }
+ else {
+ throw new IOException("Cannot access hsync() method", HSYNC_ERROR);
+ }
+ }
+ else {
+ throw new UnsupportedOperationException("hsync() is not available in this version of Hadoop.");
+ }
+ }
+
+ /**
+ * Gets the wrapped Hadoop output stream.
+ * @return The wrapped Hadoop output stream.
+ */
+ public org.apache.hadoop.fs.FSDataOutputStream getHadoopOutputStream() {
+ return fdos;
+ }
+
+ // ------------------------------------------------------------------------
+ // utilities to bridge hsync and hflush to hadoop, even through it is not supported in Hadoop 1
+ // ------------------------------------------------------------------------
+
+ private static final Method HFLUSH_METHOD;
+ private static final Method HSYNC_METHOD;
+
+ private static final Throwable HFLUSH_ERROR;
+ private static final Throwable HSYNC_ERROR;
+
+ static {
+ Method hflush = null;
+ Method hsync = null;
+
+ Throwable flushError = null;
+ Throwable syncError = null;
+
+ try {
+ hflush = org.apache.hadoop.fs.FSDataOutputStream.class.getMethod("hflush");
+ }
+ catch (Throwable t) {
+ flushError = t;
+ }
+
+ try {
+ hsync = org.apache.hadoop.fs.FSDataOutputStream.class.getMethod("hsync");
+ }
+ catch (Throwable t) {
+ syncError = t;
+ }
+
+ HFLUSH_METHOD = hflush;
+ HSYNC_METHOD = hsync;
+
+ HFLUSH_ERROR = flushError;
+ HSYNC_ERROR = syncError;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/304139d4/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
index 152b2e8..4e05ebe 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
@@ -23,18 +23,18 @@ import java.lang.reflect.Method;
import java.net.URI;
import java.net.UnknownHostException;
-import org.apache.flink.core.fs.HadoopFileSystemWrapper;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+
+import org.apache.flink.core.fs.HadoopFileSystemWrapper;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.GlobalConfiguration;
import org.apache.flink.core.fs.BlockLocation;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FSDataOutputStream;
import org.apache.flink.core.fs.FileStatus;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
import org.apache.flink.util.InstantiationUtil;
+
import org.apache.hadoop.conf.Configuration;
/**
@@ -264,6 +264,14 @@ public final class HadoopFileSystem extends FileSystem implements HadoopFileSyst
return fs.getUri();
}
+ /**
+ * Gets the underlying Hadoop FileSystem.
+ * @return The underlying Hadoop FileSystem.
+ */
+ public org.apache.hadoop.fs.FileSystem getHadoopFileSystem() {
+ return this.fs;
+ }
+
@Override
public void initialize(URI path) throws IOException {
@@ -367,21 +375,21 @@ public final class HadoopFileSystem extends FileSystem implements HadoopFileSyst
}
@Override
- public FSDataInputStream open(final Path f, final int bufferSize) throws IOException {
+ public HadoopDataInputStream open(final Path f, final int bufferSize) throws IOException {
final org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(f.toString());
final org.apache.hadoop.fs.FSDataInputStream fdis = this.fs.open(path, bufferSize);
return new HadoopDataInputStream(fdis);
}
@Override
- public FSDataInputStream open(final Path f) throws IOException {
+ public HadoopDataInputStream open(final Path f) throws IOException {
final org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(f.toString());
final org.apache.hadoop.fs.FSDataInputStream fdis = fs.open(path);
return new HadoopDataInputStream(fdis);
}
@Override
- public FSDataOutputStream create(final Path f, final boolean overwrite, final int bufferSize,
+ public HadoopDataOutputStream create(final Path f, final boolean overwrite, final int bufferSize,
final short replication, final long blockSize)
throws IOException
{
@@ -392,7 +400,7 @@ public final class HadoopFileSystem extends FileSystem implements HadoopFileSyst
@Override
- public FSDataOutputStream create(final Path f, final boolean overwrite) throws IOException {
+ public HadoopDataOutputStream create(final Path f, final boolean overwrite) throws IOException {
final org.apache.hadoop.fs.FSDataOutputStream fsDataOutputStream = this.fs
.create(new org.apache.hadoop.fs.Path(f.toString()), overwrite);
return new HadoopDataOutputStream(fsDataOutputStream);
http://git-wip-us.apache.org/repos/asf/flink/blob/304139d4/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataOutputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataOutputStream.java
index 0501364..f830613 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataOutputStream.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataOutputStream.java
@@ -69,6 +69,7 @@ public final class S3DataOutputStream extends FSDataOutputStream {
private int partNumber = 1; // First valid upload part number is 1.
private int bytesWritten = 0;
+
private final class InternalUploadInputStream extends InputStream {
@@ -83,12 +84,9 @@ public final class S3DataOutputStream extends FSDataOutputStream {
this.length = length;
}
- /**
- * {@inheritDoc}
- */
+
@Override
public int read() throws IOException {
-
if (this.length - this.bytesRead == 0) {
return -1;
}
@@ -96,21 +94,14 @@ public final class S3DataOutputStream extends FSDataOutputStream {
return (int) this.srcBuf[this.bytesRead++];
}
- /**
- * {@inheritDoc}
- */
+
@Override
public int read(final byte[] buf) throws IOException {
-
return read(buf, 0, buf.length);
}
- /**
- * {@inheritDoc}
- */
@Override
public int read(final byte[] buf, final int off, final int len) throws IOException {
-
if (this.length - this.bytesRead == 0) {
return -1;
}
@@ -122,18 +113,12 @@ public final class S3DataOutputStream extends FSDataOutputStream {
return bytesToCopy;
}
- /**
- * {@inheritDoc}
- */
@Override
public int available() throws IOException {
return (this.length - bytesRead);
}
- /**
- * {@inheritDoc}
- */
@Override
public long skip(final long n) throws IOException {
@@ -159,7 +144,6 @@ public final class S3DataOutputStream extends FSDataOutputStream {
@Override
public void write(final int b) throws IOException {
-
// Upload buffer to S3
if (this.bytesWritten == this.buf.length) {
uploadPartAndFlushBuffer();
@@ -171,11 +155,9 @@ public final class S3DataOutputStream extends FSDataOutputStream {
@Override
public void write(final byte[] b, final int off, final int len) throws IOException {
-
int nextPos = off;
while (nextPos < len) {
-
// Upload buffer to S3
if (this.bytesWritten == this.buf.length) {
uploadPartAndFlushBuffer();
@@ -191,14 +173,12 @@ public final class S3DataOutputStream extends FSDataOutputStream {
@Override
public void write(final byte[] b) throws IOException {
-
write(b, 0, b.length);
}
@Override
public void close() throws IOException {
-
if (this.uploadId == null) {
// This is not a multipart upload
@@ -251,10 +231,13 @@ public final class S3DataOutputStream extends FSDataOutputStream {
}
}
-
+ @Override
+ public void sync() throws IOException {
+ // can do nothing here
+ }
+
@Override
public void flush() throws IOException {
-
// Flush does nothing in this implementation since we ways have to transfer at least 5 MB in a multipart upload
}
@@ -267,7 +250,6 @@ public final class S3DataOutputStream extends FSDataOutputStream {
}
try {
-
if (this.partNumber >= MAX_PART_NUMBER) {
throw new IOException("Cannot upload any more data: maximum part number reached");
}
@@ -287,9 +269,11 @@ public final class S3DataOutputStream extends FSDataOutputStream {
this.bytesWritten = 0;
operationSuccessful = true;
- } catch (AmazonServiceException e) {
- throw new IOException(StringUtils.stringifyException(e));
- } finally {
+ }
+ catch (AmazonServiceException e) {
+ throw new IOException(e.getMessage(), e);
+ }
+ finally {
if (!operationSuccessful) {
abortUpload();
}
@@ -312,9 +296,11 @@ public final class S3DataOutputStream extends FSDataOutputStream {
operationSuccessful = true;
return result.getUploadId();
- } catch (AmazonServiceException e) {
- throw new IOException(StringUtils.stringifyException(e));
- } finally {
+ }
+ catch (AmazonServiceException e) {
+ throw new IOException(e.getMessage(), e);
+ }
+ finally {
if (!operationSuccessful) {
abortUpload();
}
@@ -322,7 +308,6 @@ public final class S3DataOutputStream extends FSDataOutputStream {
}
private void abortUpload() {
-
if (this.uploadId == null) {
// This is not a multipart upload, nothing to do here
return;
@@ -332,7 +317,8 @@ public final class S3DataOutputStream extends FSDataOutputStream {
final AbortMultipartUploadRequest request = new AbortMultipartUploadRequest(this.bucket, this.object,
this.uploadId);
this.s3Client.abortMultipartUpload(request);
- } catch (AmazonServiceException e) {
+ }
+ catch (AmazonServiceException e) {
// Ignore exception
}
}
[04/15] flink git commit: [FLINK-1320] [core] Add an off-heap variant
of the managed memory
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java
index 62c2013..f40171a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java
@@ -16,7 +16,6 @@
* limitations under the License.
*/
-
package org.apache.flink.runtime.operators.sort;
import java.util.List;
@@ -25,9 +24,8 @@ import java.util.Random;
import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.operators.sort.FixedLengthRecordSorter;
-import org.apache.flink.runtime.operators.sort.QuickSort;
+import org.apache.flink.core.memory.MemoryType;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.runtime.operators.testutils.RandomIntPairGenerator;
import org.apache.flink.runtime.operators.testutils.UniformIntPairGenerator;
@@ -43,15 +41,15 @@ import org.junit.Test;
/**
*
*/
-public class FixedLengthRecordSorterTest
-{
+public class FixedLengthRecordSorterTest {
+
private static final long SEED = 649180756312423613L;
private static final int MEMORY_SIZE = 1024 * 1024 * 64;
private static final int MEMORY_PAGE_SIZE = 32 * 1024;
- private DefaultMemoryManager memoryManager;
+ private MemoryManager memoryManager;
private TypeSerializer<IntPair> serializer;
@@ -60,7 +58,7 @@ public class FixedLengthRecordSorterTest
@Before
public void beforeTest() {
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, MEMORY_PAGE_SIZE);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, MEMORY_PAGE_SIZE, MemoryType.HEAP, true);
this.serializer = new IntPairSerializer();
this.comparator = new IntPairComparator();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java
index 429062b..e4dbfd3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java
@@ -39,16 +39,18 @@ import org.apache.flink.api.java.typeutils.ValueTypeInfo;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.io.disk.FileChannelOutputView;
import org.apache.flink.runtime.io.disk.SeekableFileChannelInputView;
import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.types.Value;
import org.apache.flink.util.MutableObjectIterator;
+
import org.junit.Test;
public class LargeRecordHandlerITCase {
@@ -62,7 +64,7 @@ public class LargeRecordHandlerITCase {
final int NUM_RECORDS = 10;
try {
- final DefaultMemoryManager memMan = new DefaultMemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, true);
+ final MemoryManager memMan = new MemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
final AbstractInvokable owner = new DummyInvokable();
final List<MemorySegment> initialMemory = memMan.allocatePages(owner, 6);
@@ -143,9 +145,7 @@ public class LargeRecordHandlerITCase {
fail(e.getMessage());
}
finally {
- if (ioMan != null) {
- ioMan.shutdown();
- }
+ ioMan.shutdown();
}
}
@@ -155,8 +155,6 @@ public class LargeRecordHandlerITCase {
private static final byte[] BUFFER = new byte[50000000];
-// private static final byte[] BUFFER = new byte[500000000];
-
static {
for (int i = 0; i < BUFFER.length; i++) {
BUFFER[i] = (byte) i;
@@ -178,9 +176,9 @@ public class LargeRecordHandlerITCase {
@Override
public void read(DataInputView in) throws IOException {
val = in.readInt();
- for (int i = 0; i < BUFFER.length; i++) {
+ for (byte bufferByte : BUFFER) {
byte b = in.readByte();
- assertEquals(BUFFER[i], b);
+ assertEquals(bufferByte, b);
}
}
@@ -192,7 +190,7 @@ public class LargeRecordHandlerITCase {
}
-// @Test
+ @Test
public void fileTest() {
final IOManager ioMan = new IOManagerAsync();
@@ -203,7 +201,7 @@ public class LargeRecordHandlerITCase {
FileIOChannel.ID channel = null;
try {
- final DefaultMemoryManager memMan = new DefaultMemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, true);
+ final MemoryManager memMan = new MemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
final AbstractInvokable owner = new DummyInvokable();
final List<MemorySegment> memory = memMan.allocatePages(owner, NUM_PAGES);
@@ -266,12 +264,10 @@ public class LargeRecordHandlerITCase {
if (channel != null) {
try {
ioMan.deleteChannel(channel);
- } catch (IOException e) {}
- }
-
- if (ioMan != null) {
- ioMan.shutdown();
+ } catch (IOException ignored) {}
}
+
+ ioMan.shutdown();
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java
index 423ff9a..49e1e76 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java
@@ -31,10 +31,11 @@ import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.api.java.typeutils.TypeInfoParser;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.util.MutableObjectIterator;
import org.junit.Test;
@@ -49,7 +50,7 @@ public class LargeRecordHandlerTest {
final int NUM_PAGES = 50;
try {
- final DefaultMemoryManager memMan = new DefaultMemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, true);
+ final MemoryManager memMan = new MemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
final AbstractInvokable owner = new DummyInvokable();
final List<MemorySegment> memory = memMan.allocatePages(owner, NUM_PAGES);
@@ -86,9 +87,7 @@ public class LargeRecordHandlerTest {
fail(e.getMessage());
}
finally {
- if (ioMan != null) {
- ioMan.shutdown();
- }
+ ioMan.shutdown();
}
}
@@ -101,7 +100,7 @@ public class LargeRecordHandlerTest {
final int NUM_RECORDS = 25000;
try {
- final DefaultMemoryManager memMan = new DefaultMemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, true);
+ final MemoryManager memMan = new MemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
final AbstractInvokable owner = new DummyInvokable();
final List<MemorySegment> initialMemory = memMan.allocatePages(owner, 6);
@@ -174,9 +173,7 @@ public class LargeRecordHandlerTest {
fail(e.getMessage());
}
finally {
- if (ioMan != null) {
- ioMan.shutdown();
- }
+ ioMan.shutdown();
}
}
@@ -189,7 +186,7 @@ public class LargeRecordHandlerTest {
final int NUM_RECORDS = 25000;
try {
- final DefaultMemoryManager memMan = new DefaultMemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, true);
+ final MemoryManager memMan = new MemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
final AbstractInvokable owner = new DummyInvokable();
final List<MemorySegment> initialMemory = memMan.allocatePages(owner, 6);
@@ -264,9 +261,7 @@ public class LargeRecordHandlerTest {
fail(e.getMessage());
}
finally {
- if (ioMan != null) {
- ioMan.shutdown();
- }
+ ioMan.shutdown();
}
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java
index 6548052..7f6db00 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java
@@ -32,9 +32,12 @@ import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
-import org.apache.flink.runtime.operators.testutils.*;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
+import org.apache.flink.runtime.operators.testutils.DummyInvokable;
+import org.apache.flink.runtime.operators.testutils.Match;
+import org.apache.flink.runtime.operators.testutils.MatchRemovingJoiner;
+import org.apache.flink.runtime.operators.testutils.TestData;
import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator;
import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode;
import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode;
@@ -45,7 +48,11 @@ import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
import java.util.Map.Entry;
@SuppressWarnings("deprecation")
@@ -97,7 +104,7 @@ public class NonReusingSortMergeInnerJoinIteratorITCase {
new TypeSerializer<?>[] { IntSerializer.INSTANCE });
pairComparator = new GenericPairComparator<Tuple2<Integer, String>, Tuple2<Integer, String>>(comparator1, comparator2);
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
this.ioManager = new IOManagerAsync();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java
index 91609bb..7272595 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
import org.apache.flink.util.MutableObjectIterator;
import org.junit.Test;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorterTest.java
index abead64..3e941dd 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorterTest.java
@@ -25,9 +25,8 @@ import java.util.Random;
import org.apache.flink.api.common.typeutils.record.RecordComparator;
import org.apache.flink.api.common.typeutils.record.RecordSerializer;
import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.operators.sort.NormalizedKeySorter;
-import org.apache.flink.runtime.operators.sort.QuickSort;
+import org.apache.flink.core.memory.MemoryType;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.runtime.operators.testutils.TestData;
import org.apache.flink.runtime.operators.testutils.TestData.Key;
@@ -57,12 +56,12 @@ public class NormalizedKeySorterTest
private static final int MEMORY_PAGE_SIZE = 32 * 1024;
- private DefaultMemoryManager memoryManager;
+ private MemoryManager memoryManager;
@Before
public void beforeTest() {
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, MEMORY_PAGE_SIZE);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, MEMORY_PAGE_SIZE, MemoryType.HEAP, true);
}
@After
@@ -369,8 +368,7 @@ public class NormalizedKeySorterTest
}
@Test
- public void testSortShortStringKeys() throws Exception
- {
+ public void testSortShortStringKeys() throws Exception {
final int numSegments = MEMORY_SIZE / MEMORY_PAGE_SIZE;
final List<MemorySegment> memory = this.memoryManager.allocatePages(new DummyInvokable(), numSegments);
@@ -418,8 +416,7 @@ public class NormalizedKeySorterTest
}
@Test
- public void testSortLongStringKeys() throws Exception
- {
+ public void testSortLongStringKeys() throws Exception {
final int numSegments = MEMORY_SIZE / MEMORY_PAGE_SIZE;
final List<MemorySegment> memory = this.memoryManager.allocatePages(new DummyInvokable(), numSegments);
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java
index 39316e3..d92be18 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java
@@ -32,9 +32,12 @@ import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
-import org.apache.flink.runtime.operators.testutils.*;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
+import org.apache.flink.runtime.operators.testutils.DummyInvokable;
+import org.apache.flink.runtime.operators.testutils.Match;
+import org.apache.flink.runtime.operators.testutils.MatchRemovingJoiner;
+import org.apache.flink.runtime.operators.testutils.TestData;
import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator;
import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode;
import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode;
@@ -45,7 +48,11 @@ import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
import java.util.Map.Entry;
@SuppressWarnings("deprecation")
@@ -97,7 +104,7 @@ public class ReusingSortMergeInnerJoinIteratorITCase {
new TypeSerializer<?>[] { IntSerializer.INSTANCE });
pairComparator = new GenericPairComparator<Tuple2<Integer, String>, Tuple2<Integer, String>>(comparator1, comparator2);
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
this.ioManager = new IOManagerAsync();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java
index 779cf37..2cec393 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
import org.apache.flink.util.MutableObjectIterator;
import org.junit.Test;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
index a085eeb..ece20ff 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
@@ -29,8 +29,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.PactDriver;
import org.apache.flink.runtime.operators.PactTaskContext;
import org.apache.flink.runtime.operators.ResettablePactDriver;
@@ -100,7 +99,7 @@ public class BinaryOperatorTestBase<S extends Function, IN, OUT> extends TestLog
this.perSortMem = perSortMemory;
this.perSortFractionMem = (double) perSortMemory / totalMem;
this.ioManager = new IOManagerAsync();
- this.memManager = totalMem > 0 ? new DefaultMemoryManager(totalMem, 1) : null;
+ this.memManager = totalMem > 0 ? new MemoryManager(totalMem, 1) : null;
this.inputs = new ArrayList<>();
this.comparators = new ArrayList<>();
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
index 12ca909..1737349 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
@@ -37,8 +37,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.PactDriver;
import org.apache.flink.runtime.operators.PactTaskContext;
import org.apache.flink.runtime.operators.ResettablePactDriver;
@@ -104,7 +103,7 @@ public class DriverTestBase<S extends Function> extends TestLogger implements Pa
this.perSortMem = perSortMemory;
this.perSortFractionMem = (double)perSortMemory/totalMem;
this.ioManager = new IOManagerAsync();
- this.memManager = totalMem > 0 ? new DefaultMemoryManager(totalMem,1) : null;
+ this.memManager = totalMem > 0 ? new MemoryManager(totalMem,1) : null;
this.inputs = new ArrayList<MutableObjectIterator<Record>>();
this.comparators = new ArrayList<TypeComparator<Record>>();
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
index 5fc9bb9..972aeda 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.operators.testutils;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.UnmodifiableConfiguration;
import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
import org.apache.flink.runtime.execution.Environment;
@@ -39,8 +39,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.state.StateHandle;
import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
import org.apache.flink.types.Record;
@@ -94,7 +93,7 @@ public class MockEnvironment implements Environment {
this.inputs = new LinkedList<InputGate>();
this.outputs = new LinkedList<ResultPartitionWriter>();
- this.memManager = new DefaultMemoryManager(memorySize, 1);
+ this.memManager = new MemoryManager(memorySize, 1);
this.ioManager = new IOManagerAsync();
this.inputSplitProvider = inputSplitProvider;
this.bufferSize = bufferSize;
@@ -125,7 +124,7 @@ public class MockEnvironment implements Environment {
@Override
public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
- return new Buffer(new MemorySegment(new byte[bufferSize]), mock(BufferRecycler.class));
+ return new Buffer(MemorySegmentFactory.allocateUnpooledSegment(bufferSize), mock(BufferRecycler.class));
}
});
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
index c04d051..4662762 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
@@ -29,7 +29,7 @@ import org.apache.flink.core.fs.FileSystem.WriteMode;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.io.network.partition.consumer.IteratorWrappingTestSingleInputGate;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.PactDriver;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
import org.apache.flink.runtime.operators.util.TaskConfig;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
index 9600e5a..924a16b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
@@ -29,8 +29,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.PactDriver;
import org.apache.flink.runtime.operators.PactTaskContext;
import org.apache.flink.runtime.operators.ResettablePactDriver;
@@ -106,7 +105,7 @@ public class UnaryOperatorTestBase<S extends Function, IN, OUT> extends TestLogg
this.perSortMem = perSortMemory;
this.perSortFractionMem = (double)perSortMemory/totalMem;
this.ioManager = new IOManagerAsync();
- this.memManager = totalMem > 0 ? new DefaultMemoryManager(totalMem,1) : null;
+ this.memManager = totalMem > 0 ? new MemoryManager(totalMem, 1) : null;
this.owner = new DummyInvokable();
Configuration config = new Configuration();
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/BloomFilterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/BloomFilterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/BloomFilterTest.java
index cbbeca0..8fbd4f7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/BloomFilterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/BloomFilterTest.java
@@ -18,7 +18,10 @@
package org.apache.flink.runtime.operators.util;
+import org.apache.flink.core.memory.HeapMemorySegment;
import org.apache.flink.core.memory.MemorySegment;
+
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -37,7 +40,7 @@ public class BloomFilterTest {
int bitsSize = BloomFilter.optimalNumOfBits(INPUT_SIZE, FALSE_POSITIVE_PROBABILITY);
bitsSize = bitsSize + (Long.SIZE - (bitsSize % Long.SIZE));
int byteSize = bitsSize >>> 3;
- MemorySegment memorySegment = new MemorySegment(new byte[byteSize]);
+ MemorySegment memorySegment = MemorySegmentFactory.allocateUnpooledSegment(byteSize);
bloomFilter = new BloomFilter(INPUT_SIZE, byteSize);
bloomFilter.setBitsLocation(memorySegment, 0);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
index 7debb08..1060e55 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
@@ -25,11 +25,11 @@ import org.apache.flink.api.common.typeutils.record.RecordComparator;
import org.apache.flink.api.common.typeutils.record.RecordPairComparator;
import org.apache.flink.api.common.typeutils.record.RecordSerializerFactory;
import org.apache.flink.api.java.record.functions.JoinFunction;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashMatchIterator;
import org.apache.flink.runtime.operators.hash.ReusingBuildSecondHashMatchIterator;
import org.apache.flink.runtime.operators.sort.ReusingMergeInnerJoinIterator;
@@ -94,7 +94,7 @@ public class HashVsSortMiniBenchmark {
this.comparator2 = new RecordComparator(new int[] {0}, new Class[] {TestData.Key.class});
this.pairComparator11 = new RecordPairComparator(new int[] {0}, new int[] {0}, new Class[] {TestData.Key.class});
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, PAGE_SIZE);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
this.ioManager = new IOManagerAsync();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
index 6d743d5..7857123 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
@@ -39,7 +39,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
import org.apache.flink.runtime.jobgraph.tasks.CheckpointNotificationOperator;
import org.apache.flink.runtime.jobgraph.tasks.CheckpointedOperator;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.junit.Before;
import org.junit.Test;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
index 0bb4063..0a7f08d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
@@ -27,6 +27,7 @@ import akka.actor.Props;
import akka.testkit.JavaTestKit;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.StreamingMode;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.instance.InstanceConnectionInfo;
@@ -37,8 +38,7 @@ import org.apache.flink.runtime.io.network.netty.NettyConfig;
import org.apache.flink.runtime.jobmanager.JobManager;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.messages.TaskManagerMessages;
import org.apache.flink.runtime.testingUtils.TestingUtils;
@@ -87,12 +87,12 @@ public class TaskManagerComponentsStartupShutdownTest {
config);
final NetworkEnvironmentConfiguration netConf = new NetworkEnvironmentConfiguration(
- 32, BUFFER_SIZE, IOManager.IOMode.SYNC, Option.<NettyConfig>empty(),
+ 32, BUFFER_SIZE, MemoryType.HEAP, IOManager.IOMode.SYNC, Option.<NettyConfig>empty(),
new Tuple2<Integer, Integer>(0, 0));
final InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(InetAddress.getLocalHost(), 10000);
- final MemoryManager memManager = new DefaultMemoryManager(32 * BUFFER_SIZE, 1, BUFFER_SIZE, false);
+ final MemoryManager memManager = new MemoryManager(32 * BUFFER_SIZE, 1, BUFFER_SIZE, MemoryType.HEAP, false);
final IOManager ioManager = new IOManagerAsync(TMP_DIR);
final NetworkEnvironment network = new NetworkEnvironment(
TestingUtils.defaultExecutionContext(),
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
index 5cfc2a2..cf1bfc5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
@@ -42,7 +42,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.messages.TaskMessages;
import org.junit.After;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/util/DataInputOutputSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/DataInputOutputSerializerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/DataInputOutputSerializerTest.java
index ac6744e..3631ba1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/DataInputOutputSerializerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/DataInputOutputSerializerTest.java
@@ -16,9 +16,9 @@
* limitations under the License.
*/
-
package org.apache.flink.runtime.util;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.junit.Assert;
import org.apache.flink.core.memory.MemorySegment;
@@ -38,7 +38,7 @@ public class DataInputOutputSerializerTest {
SerializationTestType randomInt = Util.randomRecord(SerializationTestTypeFactory.INT);
DataOutputSerializer serializer = new DataOutputSerializer(randomInt.length());
- MemorySegment segment = new MemorySegment(new byte[randomInt.length()]);
+ MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(randomInt.length());
try {
// empty buffer, read buffer should be empty
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
index 475115e..8746fbb 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
@@ -25,7 +25,7 @@ import org.apache.flink.runtime.instance.InstanceConnectionInfo
import org.apache.flink.runtime.io.disk.iomanager.IOManager
import org.apache.flink.runtime.io.network.NetworkEnvironment
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager
+import org.apache.flink.runtime.memory.MemoryManager
import org.apache.flink.runtime.messages.JobManagerMessages.{ResponseLeaderSessionID,
RequestLeaderSessionID}
import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect}
@@ -53,7 +53,7 @@ import scala.language.postfixOps
class TestingTaskManager(
config: TaskManagerConfiguration,
connectionInfo: InstanceConnectionInfo,
- memoryManager: DefaultMemoryManager,
+ memoryManager: MemoryManager,
ioManager: IOManager,
network: NetworkEnvironment,
numberOfSlots: Int,
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
index 2bad197..cabed14 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
@@ -27,6 +27,7 @@ import java.util.Random;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.event.AbstractEvent;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
@@ -336,7 +337,7 @@ public class BufferSpiller {
"Spilled buffer (%d bytes) is larger than page size of (%d bytes)", length, pageSize));
}
- MemorySegment seg = new MemorySegment(new byte[pageSize]);
+ MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(pageSize);
int segPos = 0;
int bytesRemaining = length;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
index 173e894..b78ec44 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
@@ -20,7 +20,8 @@
package org.apache.flink.runtime.io.network.partition.consumer;
import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.event.AbstractEvent;
import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
@@ -84,7 +85,7 @@ public class StreamTestSingleInputGate<T> extends TestSingleInputGate {
for (int i = 0; i < numInputChannels; i++) {
final int channelIndex = i;
final RecordSerializer<SerializationDelegate<Object>> recordSerializer = new SpanningRecordSerializer<SerializationDelegate<Object>>();
- final SerializationDelegate<Object> delegate = new SerializationDelegate(new MultiplexingStreamRecordSerializer<T>(serializer));
+ final SerializationDelegate<Object> delegate = new SerializationDelegate<Object>(new MultiplexingStreamRecordSerializer<T>(serializer));
inputQueues[channelIndex] = new ConcurrentLinkedQueue<InputValue<Object>>();
inputChannels[channelIndex] = new TestInputChannel(inputGate, i);
@@ -98,20 +99,25 @@ public class StreamTestSingleInputGate<T> extends TestSingleInputGate {
when(inputChannels[channelIndex].getInputChannel().isReleased()).thenReturn(
true);
return EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE);
- } else if (input != null && input.isStreamRecord()) {
+ }
+ else if (input != null && input.isStreamRecord()) {
Object inputElement = input.getStreamRecord();
- final Buffer buffer = new Buffer(new MemorySegment(new byte[bufferSize]),
+ final Buffer buffer = new Buffer(
+ MemorySegmentFactory.allocateUnpooledSegment(bufferSize),
mock(BufferRecycler.class));
+
recordSerializer.setNextBuffer(buffer);
delegate.setInstance(inputElement);
recordSerializer.addRecord(delegate);
// Call getCurrentBuffer to ensure size is set
return recordSerializer.getCurrentBuffer();
- } else if (input != null && input.isEvent()) {
+ }
+ else if (input != null && input.isEvent()) {
AbstractEvent event = input.getEvent();
return EventSerializer.toBuffer(event);
- } else {
+ }
+ else {
synchronized (inputQueues[channelIndex]) {
inputQueues[channelIndex].wait();
return answer(invocationOnMock);
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
index 678b145..a8a989b 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.Random;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.event.TaskEvent;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
@@ -50,8 +51,8 @@ public class BarrierBufferMassiveRandomTest {
try {
ioMan = new IOManagerAsync();
- BufferPool pool1 = new NetworkBufferPool(100, PAGE_SIZE).createBufferPool(100, true);
- BufferPool pool2 = new NetworkBufferPool(100, PAGE_SIZE).createBufferPool(100, true);
+ BufferPool pool1 = new NetworkBufferPool(100, PAGE_SIZE, MemoryType.HEAP).createBufferPool(100, true);
+ BufferPool pool2 = new NetworkBufferPool(100, PAGE_SIZE, MemoryType.HEAP).createBufferPool(100, true);
RandomGeneratingInputGate myIG = new RandomGeneratingInputGate(
new BufferPool[] { pool1, pool2 },
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java
index a95839a..d4fdc59 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java
@@ -18,7 +18,8 @@
package org.apache.flink.streaming.runtime.io;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
@@ -888,7 +889,10 @@ public class BarrierBufferTest {
private static BufferOrEvent createBuffer(int channel) {
// since we have no access to the contents, we need to use the size as an
// identifier to validate correctness here
- Buffer buf = new Buffer(new MemorySegment(new byte[PAGE_SIZE]), FreeingBufferRecycler.INSTANCE);
+ Buffer buf = new Buffer(
+ MemorySegmentFactory.allocateUnpooledSegment(PAGE_SIZE),
+ FreeingBufferRecycler.INSTANCE);
+
buf.setSize(SIZE_COUNTER++);
return new BufferOrEvent(buf, channel);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
index c6010d6..b9b6e5f 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
@@ -18,7 +18,7 @@
package org.apache.flink.streaming.runtime.io;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
@@ -339,7 +339,7 @@ public class BarrierTrackerTest {
private static BufferOrEvent createBuffer(int channel) {
return new BufferOrEvent(
- new Buffer(new MemorySegment(new byte[] { 1, 2 }), FreeingBufferRecycler.INSTANCE), channel);
+ new Buffer(MemorySegmentFactory.wrap(new byte[]{1, 2}), FreeingBufferRecycler.INSTANCE), channel);
}
// ------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
index 355b7c8..e85eddb 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
@@ -19,6 +19,7 @@
package org.apache.flink.streaming.runtime.io;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.io.network.buffer.Buffer;
@@ -347,7 +348,7 @@ public class BufferSpillerTest {
}
private static BufferOrEvent generateRandomBuffer(int size, int channelIndex) {
- MemorySegment seg = new MemorySegment(new byte[PAGE_SIZE]);
+ MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(PAGE_SIZE);
for (int i = 0; i < size; i++) {
seg.put(i, (byte) i);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java
index 7cc1958..45bbbda 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java
@@ -19,7 +19,7 @@
package org.apache.flink.streaming.runtime.io;
import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.network.api.writer.ChannelSelector;
import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
import org.apache.flink.runtime.io.network.api.writer.RoundRobinChannelSelector;
@@ -33,7 +33,6 @@ import org.junit.runner.RunWith;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
-
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
@@ -95,7 +94,9 @@ public class StreamRecordWriterTest {
when(mockProvider.requestBufferBlocking()).thenAnswer(new Answer<Buffer>() {
@Override
public Buffer answer(InvocationOnMock invocation) {
- return new Buffer(new MemorySegment(new byte[4096]), FreeingBufferRecycler.INSTANCE);
+ return new Buffer(
+ MemorySegmentFactory.allocateUnpooledSegment(4096),
+ FreeingBufferRecycler.INSTANCE);
}
});
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
index 9091fa7..4fec118 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.UnmodifiableConfiguration;
import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
import org.apache.flink.runtime.event.AbstractEvent;
@@ -40,8 +40,7 @@ import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
import org.apache.flink.runtime.plugable.DeserializationDelegate;
import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate;
@@ -95,7 +94,7 @@ public class StreamMockEnvironment implements Environment {
this.inputs = new LinkedList<InputGate>();
this.outputs = new LinkedList<ResultPartitionWriter>();
- this.memManager = new DefaultMemoryManager(memorySize, 1);
+ this.memManager = new MemoryManager(memorySize, 1);
this.ioManager = new IOManagerAsync();
this.inputSplitProvider = inputSplitProvider;
this.bufferSize = bufferSize;
@@ -117,7 +116,9 @@ public class StreamMockEnvironment implements Environment {
@Override
public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
- return new Buffer(new MemorySegment(new byte[bufferSize]), mock(BufferRecycler.class));
+ return new Buffer(
+ MemorySegmentFactory.allocateUnpooledSegment(bufferSize),
+ mock(BufferRecycler.class));
}
});
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
index 987d0bc..96dbeab 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
@@ -24,7 +24,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.event.AbstractEvent;
import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
import org.apache.flink.streaming.api.collector.selector.BroadcastOutputSelectorWrapper;
import org.apache.flink.streaming.api.collector.selector.OutputSelector;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezRuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezRuntimeEnvironment.java b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezRuntimeEnvironment.java
index 889255c..b61a9b6 100644
--- a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezRuntimeEnvironment.java
+++ b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezRuntimeEnvironment.java
@@ -18,22 +18,19 @@
package org.apache.flink.tez.runtime;
-
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
public class TezRuntimeEnvironment {
-
- private static int DEFAULT_PAGE_SIZE = 32768;
-
+
private final IOManager ioManager;
private final MemoryManager memoryManager;
public TezRuntimeEnvironment(long totalMemory) {
- this.memoryManager = new DefaultMemoryManager(totalMemory, 1, DefaultMemoryManager.DEFAULT_PAGE_SIZE, true);
+ this.memoryManager = new MemoryManager(totalMemory, 1, MemoryManager.DEFAULT_PAGE_SIZE, MemoryType.HEAP, true);
this.ioManager = new IOManagerAsync();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java
index 47fbad7..a745177 100644
--- a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java
+++ b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java
@@ -35,7 +35,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.PactDriver;
import org.apache.flink.runtime.operators.PactTaskContext;
import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-tests/src/test/java/org/apache/flink/test/manual/HashTableRecordWidthCombinations.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/HashTableRecordWidthCombinations.java b/flink-tests/src/test/java/org/apache/flink/test/manual/HashTableRecordWidthCombinations.java
index 493de07..cb20e72 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/HashTableRecordWidthCombinations.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/HashTableRecordWidthCombinations.java
@@ -28,6 +28,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.runtime.TupleComparator;
import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.operators.hash.MutableHashTable;
@@ -183,7 +184,7 @@ public class HashTableRecordWidthCombinations {
private static List<MemorySegment> getMemory(int numSegments, int segmentSize) {
ArrayList<MemorySegment> list = new ArrayList<MemorySegment>(numSegments);
for (int i = 0; i < numSegments; i++) {
- list.add(new MemorySegment(new byte[segmentSize]));
+ list.add(MemorySegmentFactory.allocateUnpooledSegment(segmentSize));
}
return list;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java
index 2da29e8..c11b93c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java
@@ -27,7 +27,6 @@ import java.io.IOException;
import java.util.Random;
import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.base.StringComparator;
@@ -38,8 +37,7 @@ import org.apache.flink.api.java.typeutils.TypeInfoParser;
import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.util.MutableObjectIterator;
@@ -83,7 +81,7 @@ public class MassiveStringSorting {
BufferedReader verifyReader = null;
try {
- MemoryManager mm = new DefaultMemoryManager(1024 * 1024, 1);
+ MemoryManager mm = new MemoryManager(1024 * 1024, 1);
IOManager ioMan = new IOManagerAsync();
TypeSerializer<String> serializer = StringSerializer.INSTANCE;
@@ -173,7 +171,7 @@ public class MassiveStringSorting {
BufferedReader verifyReader = null;
try {
- MemoryManager mm = new DefaultMemoryManager(1024 * 1024, 1);
+ MemoryManager mm = new MemoryManager(1024 * 1024, 1);
IOManager ioMan = new IOManagerAsync();
TupleTypeInfo<Tuple2<String, String[]>> typeInfo = (TupleTypeInfo<Tuple2<String, String[]>>)
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java
index bc65ab9..7a484e7 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java
@@ -37,8 +37,7 @@ import org.apache.flink.api.java.typeutils.runtime.CopyableValueSerializer;
import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.types.StringValue;
@@ -82,7 +81,7 @@ public class MassiveStringValueSorting {
BufferedReader verifyReader = null;
try {
- MemoryManager mm = new DefaultMemoryManager(1024 * 1024, 1);
+ MemoryManager mm = new MemoryManager(1024 * 1024, 1);
IOManager ioMan = new IOManagerAsync();
TypeSerializer<StringValue> serializer = new CopyableValueSerializer<StringValue>(StringValue.class);
@@ -129,9 +128,11 @@ public class MassiveStringValueSorting {
}
finally {
if (input != null) {
+ //noinspection ResultOfMethodCallIgnored
input.delete();
}
if (sorted != null) {
+ //noinspection ResultOfMethodCallIgnored
sorted.delete();
}
}
@@ -173,7 +174,7 @@ public class MassiveStringValueSorting {
BufferedReader verifyReader = null;
try {
- MemoryManager mm = new DefaultMemoryManager(1024 * 1024, 1);
+ MemoryManager mm = new MemoryManager(1024 * 1024, 1);
IOManager ioMan = new IOManagerAsync();
TupleTypeInfo<Tuple2<StringValue, StringValue[]>> typeInfo = (TupleTypeInfo<Tuple2<StringValue, StringValue[]>>)
@@ -251,9 +252,11 @@ public class MassiveStringValueSorting {
}
finally {
if (input != null) {
+ //noinspection ResultOfMethodCallIgnored
input.delete();
}
if (sorted != null) {
+ //noinspection ResultOfMethodCallIgnored
sorted.delete();
}
}
@@ -348,7 +351,9 @@ public class MassiveStringValueSorting {
wrt.newLine();
}
} finally {
- wrt.close();
+ if (wrt != null) {
+ wrt.close();
+ }
}
return f;
@@ -388,7 +393,9 @@ public class MassiveStringValueSorting {
wrt.newLine();
}
} finally {
- wrt.close();
+ if (wrt != null) {
+ wrt.close();
+ }
}
return f;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala
index 157aa0d..dd27eb5 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala
@@ -28,7 +28,7 @@ import java.io.BufferedReader
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync
import java.io.FileReader
import org.apache.flink.util.MutableObjectIterator
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager
+import org.apache.flink.runtime.memory.MemoryManager
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.common.typeutils.CompositeType
import org.apache.flink.runtime.operators.sort.UnilateralSortMerger
@@ -89,7 +89,7 @@ class MassiveCaseClassSortingITCase {
0,
new ExecutionConfig)
- val mm = new DefaultMemoryManager(1024 * 1024, 1)
+ val mm = new MemoryManager(1024 * 1024, 1)
val ioMan = new IOManagerAsync()
sorter = new UnilateralSortMerger[StringTuple](mm, ioMan, inputIterator,
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/CaseClassComparatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/CaseClassComparatorTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/CaseClassComparatorTest.scala
index 21c6581..1b48612 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/CaseClassComparatorTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/CaseClassComparatorTest.scala
@@ -18,25 +18,19 @@
package org.apache.flink.api.scala.runtime
+import java.util
+import java.util.{ArrayList, List, Random}
+
import org.apache.flink.api.common.ExecutionConfig
-import org.junit.Test
-import org.junit.Assert._
-import org.apache.flink.api.scala._
-import org.apache.flink.api.common.typeutils.TypeSerializer
import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.core.memory.DataInputView
-import java.io.IOException
-import org.apache.flink.api.common.typeutils.TypeComparator
-import com.amazonaws.services.sqs.model.UnsupportedOperationException
-import org.apache.flink.core.memory.MemorySegment
-import org.apache.flink.core.memory.DataOutputView
+import org.apache.flink.api.common.typeutils.{CompositeType, TypeComparator}
+import org.apache.flink.api.scala._
+import org.apache.flink.core.memory._
+import org.apache.flink.runtime.operators.sort.{NormalizedKeySorter, QuickSort}
+
+import org.junit.Assert._
+import org.junit.Test
import org.mockito.Mockito
-import org.apache.flink.runtime.operators.sort.NormalizedKeySorter
-import java.util.List
-import java.util.ArrayList
-import org.apache.flink.runtime.operators.sort.QuickSort
-import java.util.Random
class CaseClassComparatorTest {
@@ -78,9 +72,9 @@ class CaseClassComparatorTest {
val numMemSegs = 20
- val memory : List[MemorySegment] = new ArrayList[MemorySegment](numMemSegs)
+ val memory : util.List[MemorySegment] = new util.ArrayList[MemorySegment](numMemSegs)
for (i <- 1 to numMemSegs) {
- memory.add(new MemorySegment(new Array[Byte](32*1024)))
+ memory.add(MemorySegmentFactory.allocateUnpooledSegment(32*1024))
}
val sorter : NormalizedKeySorter[CaseTestClass] = new NormalizedKeySorter[CaseTestClass](
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
index b95eb86..c9e193f 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
@@ -22,7 +22,7 @@ import org.apache.flink.runtime.instance.InstanceConnectionInfo
import org.apache.flink.runtime.io.disk.iomanager.IOManager
import org.apache.flink.runtime.io.network.NetworkEnvironment
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager
+import org.apache.flink.runtime.memory.MemoryManager
import org.apache.flink.runtime.taskmanager.{NetworkEnvironmentConfiguration, TaskManagerConfiguration, TaskManager}
import org.apache.flink.yarn.Messages.StopYarnSession
@@ -32,7 +32,7 @@ import org.apache.flink.yarn.Messages.StopYarnSession
class YarnTaskManager(
config: TaskManagerConfiguration,
connectionInfo: InstanceConnectionInfo,
- memoryManager: DefaultMemoryManager,
+ memoryManager: MemoryManager,
ioManager: IOManager,
network: NetworkEnvironment,
numberOfSlots: Int,
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index cdee6f6..7f17396 100644
--- a/pom.xml
+++ b/pom.xml
@@ -803,7 +803,7 @@ under the License.
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-checkstyle-plugin</artifactId>
- <version>2.12.1</version>
+ <version>2.16</version>
<executions>
<execution>
<id>validate</id>
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/tools/maven/checkstyle.xml
----------------------------------------------------------------------
diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml
index a2864a0..ecedb19 100644
--- a/tools/maven/checkstyle.xml
+++ b/tools/maven/checkstyle.xml
@@ -52,7 +52,7 @@ under the License.
<module name="Regexp">
<property name="format" value="org\.apache\.commons\.lang3\.Validate"/>
<property name="illegalPattern" value="true"/>
- <property name="message" value="Use Guava Checks instead of Commons Validate. Please refer coding guidelines."/>
+ <property name="message" value="Use Guava Checks instead of Commons Validate. Please refer to the coding guidelines."/>
</module>
<module name="NeedBraces">
<property name="tokens" value="LITERAL_IF, LITERAL_ELSE"/>
[06/15] flink git commit: [FLINK-1320] [core] Add an off-heap variant
of the managed memory
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
index 529d3d1..f11b933 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
@@ -37,9 +37,10 @@ import com.fasterxml.jackson.databind.ObjectMapper
import grizzled.slf4j.Logger
import org.apache.flink.configuration._
-
+import org.apache.flink.core.memory.{HybridMemorySegment, HeapMemorySegment, MemorySegmentFactory, MemoryType}
import org.apache.flink.runtime.accumulators.AccumulatorSnapshot
import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalListener, LeaderRetrievalService}
+import org.apache.flink.runtime.memory.MemoryManager.HeapMemoryPool
import org.apache.flink.runtime.messages.TaskMessages._
import org.apache.flink.runtime.messages.checkpoint.{NotifyCheckpointComplete, TriggerCheckpoint, AbstractCheckpointMessage}
import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMessages, StreamingMode}
@@ -50,14 +51,13 @@ import org.apache.flink.runtime.deployment.{InputChannelDeploymentDescriptor, Ta
import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, FallbackLibraryCacheManager, LibraryCacheManager}
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID
import org.apache.flink.runtime.filecache.FileCache
-import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription,
-InstanceConnectionInfo, InstanceID}
+import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription, InstanceConnectionInfo, InstanceID}
import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode
import org.apache.flink.runtime.io.disk.iomanager.{IOManager, IOManagerAsync}
import org.apache.flink.runtime.io.network.NetworkEnvironment
import org.apache.flink.runtime.io.network.netty.NettyConfig
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID
-import org.apache.flink.runtime.memorymanager.{MemoryManager, DefaultMemoryManager}
+import org.apache.flink.runtime.memory.MemoryManager
import org.apache.flink.runtime.messages.Messages._
import org.apache.flink.runtime.messages.RegistrationMessages._
import org.apache.flink.runtime.messages.TaskManagerMessages._
@@ -136,7 +136,7 @@ class TaskManager(
protected val askTimeout = new Timeout(config.timeout)
/** The TaskManager's physical execution resources */
- protected val resources = HardwareDescription.extractFromSystem(memoryManager.getMemorySize)
+ protected val resources = HardwareDescription.extractFromSystem(memoryManager.getMemorySize())
/** Registry of all tasks currently executed by this TaskManager */
protected val runningTasks = new java.util.HashMap[ExecutionAttemptID, Task]()
@@ -1548,7 +1548,8 @@ object TaskManager {
val (taskManagerConfig : TaskManagerConfiguration,
netConfig: NetworkEnvironmentConfiguration,
- connectionInfo: InstanceConnectionInfo
+ connectionInfo: InstanceConnectionInfo,
+ memType: MemoryType
) = parseTaskManagerConfiguration(
configuration,
taskManagerHostname,
@@ -1577,7 +1578,7 @@ object TaskManager {
LOG.info(s"Using $configuredMemory MB for Flink managed memory.")
configuredMemory << 20 // megabytes to bytes
}
- else {
+ else if (memType == MemoryType.HEAP) {
val fraction = configuration.getFloat(
ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION)
@@ -1589,7 +1590,24 @@ object TaskManager {
fraction).toLong
LOG.info(s"Using $fraction of the currently free heap space for Flink managed " +
- s"memory (${relativeMemSize >> 20} MB).")
+ s" heap memory (${relativeMemSize >> 20} MB).")
+
+ relativeMemSize
+ }
+ else {
+ val ratio = configuration.getFloat(
+ ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_RATIO_KEY,
+ ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_OFF_HEAP_RATIO)
+
+ checkConfigParameter(ratio > 0.0f,
+ ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_RATIO_KEY,
+ "MemoryManager ratio (off-heap memory / heap size) must be larger than zero")
+
+ val maxHeapSize = EnvironmentInformation.getMaxJvmHeapMemory()
+ val relativeMemSize = (maxHeapSize * ratio).toLong
+
+ LOG.info(s"Using $ratio time the heap size (${maxHeapSize} bytes) for Flink " +
+ s"managed off-heap memory (${relativeMemSize >> 20} MB).")
relativeMemSize
}
@@ -1598,16 +1616,27 @@ object TaskManager {
// now start the memory manager
val memoryManager = try {
- new DefaultMemoryManager(
+ new MemoryManager(
memorySize,
taskManagerConfig.numberOfSlots,
netConfig.networkBufferSize,
+ memType,
preAllocateMemory)
}
catch {
- case e: OutOfMemoryError => throw new Exception(
- "OutOfMemory error (" + e.getMessage + ") while allocating the TaskManager memory (" +
- memorySize + " bytes).", e)
+ case e: OutOfMemoryError =>
+ memType match {
+ case MemoryType.HEAP =>
+ throw new Exception(s"OutOfMemory error (${e.getMessage()})" +
+ s" while allocating the TaskManager heap memory (${memorySize} bytes).", e)
+
+ case MemoryType.OFF_HEAP =>
+ throw new Exception(s"OutOfMemory error (${e.getMessage()})" +
+ s" while allocating the TaskManager off-heap memory (${memorySize} bytes). " +
+ s"Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e)
+
+ case _ => throw e
+ }
}
// start the I/O manager last, it will create some temp directories.
@@ -1692,7 +1721,8 @@ object TaskManager {
localTaskManagerCommunication: Boolean)
: (TaskManagerConfiguration,
NetworkEnvironmentConfiguration,
- InstanceConnectionInfo) = {
+ InstanceConnectionInfo,
+ MemoryType) = {
// ------- read values from the config and check them ---------
// (a lot of them)
@@ -1738,9 +1768,9 @@ object TaskManager {
val pageSize: Int =
if (pageSizeNew != -1) {
// new page size has been configured
- checkConfigParameter(pageSizeNew >= DefaultMemoryManager.MIN_PAGE_SIZE, pageSizeNew,
+ checkConfigParameter(pageSizeNew >= MemoryManager.MIN_PAGE_SIZE, pageSizeNew,
ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
- "Minimum memory segment size is " + DefaultMemoryManager.MIN_PAGE_SIZE)
+ "Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE)
checkConfigParameter(MathUtils.isPowerOf2(pageSizeNew), pageSizeNew,
ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
@@ -1754,9 +1784,9 @@ object TaskManager {
}
else {
// old page size has been configured
- checkConfigParameter(pageSizeOld >= DefaultMemoryManager.MIN_PAGE_SIZE, pageSizeOld,
+ checkConfigParameter(pageSizeOld >= MemoryManager.MIN_PAGE_SIZE, pageSizeOld,
ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
- "Minimum buffer size is " + DefaultMemoryManager.MIN_PAGE_SIZE)
+ "Minimum buffer size is " + MemoryManager.MIN_PAGE_SIZE)
checkConfigParameter(MathUtils.isPowerOf2(pageSizeOld), pageSizeOld,
ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
@@ -1765,6 +1795,35 @@ object TaskManager {
pageSizeOld
}
+ // check whether we use heap or off-heap memory
+ val memType: MemoryType =
+ if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
+ MemoryType.OFF_HEAP
+ } else {
+ MemoryType.HEAP
+ }
+
+ // initialize the memory segment factory accordingly
+ memType match {
+ case MemoryType.HEAP =>
+ if (!MemorySegmentFactory.isInitialized()) {
+ MemorySegmentFactory.initializeFactory(HeapMemorySegment.FACTORY)
+ }
+ else if (MemorySegmentFactory.getFactory() != HeapMemorySegment.FACTORY) {
+ throw new Exception("Memory type is set to heap memory, but memory segment " +
+ "factory has been initialized for off-heap memory segments")
+ }
+
+ case MemoryType.OFF_HEAP =>
+ if (!MemorySegmentFactory.isInitialized()) {
+ MemorySegmentFactory.initializeFactory(HybridMemorySegment.FACTORY)
+ }
+ else if (MemorySegmentFactory.getFactory() != HybridMemorySegment.FACTORY) {
+ throw new Exception("Memory type is set to off-heap memory, but memory segment " +
+ "factory has been initialized for heap memory segments")
+ }
+ }
+
val tmpDirs = configuration.getString(
ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH)
@@ -1783,7 +1842,8 @@ object TaskManager {
}
// Default spill I/O mode for intermediate results
- val syncOrAsync = configuration.getString(ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
+ val syncOrAsync = configuration.getString(
+ ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE)
val ioMode : IOMode = if (syncOrAsync == "async") IOMode.ASYNC else IOMode.SYNC
@@ -1791,6 +1851,7 @@ object TaskManager {
val networkConfig = NetworkEnvironmentConfiguration(
numNetworkBuffers,
pageSize,
+ memType,
ioMode,
nettyConfig)
@@ -1834,7 +1895,7 @@ object TaskManager {
slots,
configuration)
- (taskManagerConfig, networkConfig, connectionInfo)
+ (taskManagerConfig, networkConfig, connectionInfo, memType)
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java
index f2a5e2d..a44916a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java
@@ -22,6 +22,7 @@ package org.apache.flink.runtime.io.disk;
import java.io.EOFException;
import java.util.List;
+import org.apache.flink.core.memory.MemoryType;
import org.junit.Assert;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader;
@@ -32,8 +33,7 @@ import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.runtime.operators.testutils.TestData;
import org.apache.flink.runtime.operators.testutils.TestData.Key;
@@ -78,7 +78,7 @@ public class ChannelViewsTest
@Before
public void beforeTest() {
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1, MEMORY_PAGE_SIZE, true);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, MEMORY_PAGE_SIZE, MemoryType.HEAP, true);
this.ioManager = new IOManagerAsync();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java
index dcc1e5f..5deb50e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.io.disk;
import static org.junit.Assert.*;
+import org.apache.flink.core.memory.MemoryType;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -29,8 +30,7 @@ import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter;
import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.runtime.operators.testutils.PairGenerator;
import org.apache.flink.runtime.operators.testutils.PairGenerator.KeyMode;
@@ -66,7 +66,8 @@ public class FileChannelStreamsITCase {
@Before
public void beforeTest() {
- memManager = new DefaultMemoryManager(NUM_MEMORY_SEGMENTS * MEMORY_PAGE_SIZE, 1, MEMORY_PAGE_SIZE, true);
+ memManager = new MemoryManager(NUM_MEMORY_SEGMENTS * MEMORY_PAGE_SIZE, 1,
+ MEMORY_PAGE_SIZE, MemoryType.HEAP, true);
ioManager = new IOManagerAsync();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java
index f9b8b38..1c2b3de 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java
@@ -26,13 +26,13 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader;
import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter;
import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.types.StringValue;
import org.junit.Test;
@@ -44,7 +44,7 @@ public class FileChannelStreamsTest {
public void testCloseAndDeleteOutputView() {
final IOManager ioManager = new IOManagerAsync();
try {
- MemoryManager memMan = new DefaultMemoryManager(4 * 16*1024, 1, 16*1024, true);
+ MemoryManager memMan = new MemoryManager(4 * 16*1024, 1, 16*1024, MemoryType.HEAP, true);
List<MemorySegment> memory = new ArrayList<MemorySegment>();
memMan.allocatePages(new DummyInvokable(), memory, 4);
@@ -78,7 +78,7 @@ public class FileChannelStreamsTest {
public void testCloseAndDeleteInputView() {
final IOManager ioManager = new IOManagerAsync();
try {
- MemoryManager memMan = new DefaultMemoryManager(4 * 16*1024, 1, 16*1024, true);
+ MemoryManager memMan = new MemoryManager(4 * 16*1024, 1, 16*1024, MemoryType.HEAP, true);
List<MemorySegment> memory = new ArrayList<MemorySegment>();
memMan.allocatePages(new DummyInvokable(), memory, 4);
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java
index c071bef..4c6a2b3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java
@@ -25,12 +25,12 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter;
import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.junit.Test;
@@ -45,7 +45,7 @@ public class SeekableFileChannelInputViewTest {
// integers across 7.x pages (7 pages = 114.688 bytes, 8 pages = 131.072 bytes)
try {
- MemoryManager memMan = new DefaultMemoryManager(4 * PAGE_SIZE, 1, PAGE_SIZE, true);
+ MemoryManager memMan = new MemoryManager(4 * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
List<MemorySegment> memory = new ArrayList<MemorySegment>();
memMan.allocatePages(new DummyInvokable(), memory, 4);
@@ -71,7 +71,7 @@ public class SeekableFileChannelInputViewTest {
try {
in.readInt();
fail("should throw EOF exception");
- } catch (EOFException e) {}
+ } catch (EOFException ignored) {}
// seek to the middle of the 3rd page
int i = 2 * PAGE_SIZE + PAGE_SIZE / 4;
@@ -82,7 +82,7 @@ public class SeekableFileChannelInputViewTest {
try {
in.readInt();
fail("should throw EOF exception");
- } catch (EOFException e) {}
+ } catch (EOFException ignored) {}
// seek to the end
i = 120000 - 4;
@@ -93,7 +93,7 @@ public class SeekableFileChannelInputViewTest {
try {
in.readInt();
fail("should throw EOF exception");
- } catch (EOFException e) {}
+ } catch (EOFException ignored) {}
// seek to the beginning
i = 0;
@@ -104,7 +104,7 @@ public class SeekableFileChannelInputViewTest {
try {
in.readInt();
fail("should throw EOF exception");
- } catch (EOFException e) {}
+ } catch (EOFException ignored) {}
// seek to after a page
i = PAGE_SIZE;
@@ -115,7 +115,7 @@ public class SeekableFileChannelInputViewTest {
try {
in.readInt();
fail("should throw EOF exception");
- } catch (EOFException e) {}
+ } catch (EOFException ignored) {}
// seek to after a page
i = 3 * PAGE_SIZE;
@@ -126,7 +126,7 @@ public class SeekableFileChannelInputViewTest {
try {
in.readInt();
fail("should throw EOF exception");
- } catch (EOFException e) {}
+ } catch (EOFException ignored) {}
// seek to the end
i = NUM_RECORDS;
@@ -134,17 +134,17 @@ public class SeekableFileChannelInputViewTest {
try {
in.readInt();
fail("should throw EOF exception");
- } catch (EOFException e) {}
+ } catch (EOFException ignored) {}
// seek out of bounds
try {
in.seek(-10);
fail("should throw an exception");
- } catch (IllegalArgumentException e) {}
+ } catch (IllegalArgumentException ignored) {}
try {
in.seek(NUM_RECORDS + 1);
fail("should throw an exception");
- } catch (IllegalArgumentException e) {}
+ } catch (IllegalArgumentException ignored) {}
}
catch (Exception e) {
e.printStackTrace();
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java
index 6a9a20a..0b1e0c3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java
@@ -16,7 +16,6 @@
* limitations under the License.
*/
-
package org.apache.flink.runtime.io.disk;
import org.apache.flink.core.memory.DataInputView;
@@ -24,9 +23,8 @@ import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.runtime.operators.testutils.TestData;
import org.apache.flink.runtime.operators.testutils.TestData.Generator.KeyMode;
@@ -34,6 +32,7 @@ import org.apache.flink.runtime.operators.testutils.TestData.Generator.ValueMode
import org.apache.flink.runtime.operators.testutils.TestData.Key;
import org.apache.flink.runtime.operators.testutils.TestData.Value;
import org.apache.flink.types.Record;
+
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -42,7 +41,6 @@ import org.junit.Test;
import java.io.EOFException;
import java.util.ArrayList;
-
public class SpillingBufferTest {
private static final long SEED = 649180756312423613L;
@@ -69,7 +67,7 @@ public class SpillingBufferTest {
@Before
public void beforeTest() {
- memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+ memoryManager = new MemoryManager(MEMORY_SIZE, 1);
ioManager = new IOManagerAsync();
}
@@ -91,8 +89,7 @@ public class SpillingBufferTest {
// --------------------------------------------------------------------------------------------
@Test
- public void testWriteReadInMemory() throws Exception
- {
+ public void testWriteReadInMemory() throws Exception {
final TestData.Generator generator = new TestData.Generator(SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
// create the writer output view
@@ -150,9 +147,9 @@ public class SpillingBufferTest {
}
@Test
- public void testWriteReadTooMuchInMemory() throws Exception
- {
- final TestData.Generator generator = new TestData.Generator(SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+ public void testWriteReadTooMuchInMemory() throws Exception {
+ final TestData.Generator generator = new TestData.Generator(
+ SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
// create the writer output view
final ArrayList<MemorySegment> memory = new ArrayList<MemorySegment>(NUM_MEMORY_SEGMENTS);
@@ -217,9 +214,9 @@ public class SpillingBufferTest {
// --------------------------------------------------------------------------------------------
@Test
- public void testWriteReadExternal() throws Exception
- {
- final TestData.Generator generator = new TestData.Generator(SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+ public void testWriteReadExternal() throws Exception {
+ final TestData.Generator generator = new TestData.Generator(
+ SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
// create the writer output view
final ArrayList<MemorySegment> memory = new ArrayList<MemorySegment>(NUM_MEMORY_SEGMENTS);
@@ -276,9 +273,9 @@ public class SpillingBufferTest {
}
@Test
- public void testWriteReadTooMuchExternal() throws Exception
- {
- final TestData.Generator generator = new TestData.Generator(SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+ public void testWriteReadTooMuchExternal() throws Exception {
+ final TestData.Generator generator = new TestData.Generator(
+ SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
// create the writer output view
final ArrayList<MemorySegment> memory = new ArrayList<MemorySegment>(NUM_MEMORY_SEGMENTS);
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java
index 49e93c6..a471e66 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java
@@ -18,7 +18,9 @@
package org.apache.flink.runtime.io.disk.iomanager;
+import org.apache.flink.core.memory.HeapMemorySegment;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.util.TestNotificationListener;
import org.junit.Test;
@@ -274,7 +276,7 @@ public class AsynchronousFileIOChannelTest {
try {
final int NUM_BLOCKS = 100;
- final MemorySegment seg = new MemorySegment(new byte[32 * 1024]);
+ final MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(32 * 1024);
final AtomicInteger callbackCounter = new AtomicInteger();
final AtomicBoolean exceptionOccurred = new AtomicBoolean();
@@ -336,7 +338,7 @@ public class AsynchronousFileIOChannelTest {
private void testExceptionForwardsToClose(IOManagerAsync ioMan, final int numBlocks, final int failingBlock) {
try {
- MemorySegment seg = new MemorySegment(new byte[32 * 1024]);
+ MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(32 * 1024);
FileIOChannel.ID channelId = ioMan.createChannel();
BlockChannelWriterWithCallback<MemorySegment> writer = new AsynchronousBlockWriterWithCallback(channelId,
@@ -371,7 +373,7 @@ public class AsynchronousFileIOChannelTest {
finally {
try {
writer.closeAndDelete();
- } catch (Throwable t) {}
+ } catch (Throwable ignored) {}
}
}
catch (Exception e) {
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java
index 294a6e6..c1bd465 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java
@@ -19,6 +19,7 @@
package org.apache.flink.runtime.io.disk.iomanager;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
import org.apache.flink.runtime.testutils.DiscardingRecycler;
@@ -146,7 +147,9 @@ public class BufferFileWriterFileSegmentReaderTest {
fileSegment.getFileChannel().read(buffer, fileSegment.getPosition());
- currentNumber = verifyBufferFilledWithAscendingNumbers(new Buffer(new MemorySegment(buffer.array()), BUFFER_RECYCLER), currentNumber, fileSegment.getLength());
+ currentNumber = verifyBufferFilledWithAscendingNumbers(
+ new Buffer(MemorySegmentFactory.wrap(buffer.array()), BUFFER_RECYCLER),
+ currentNumber, fileSegment.getLength());
}
reader.close();
@@ -169,7 +172,7 @@ public class BufferFileWriterFileSegmentReaderTest {
}
private Buffer createBuffer() {
- return new Buffer(new MemorySegment(new byte[BUFFER_SIZE]), BUFFER_RECYCLER);
+ return new Buffer(MemorySegmentFactory.allocateUnpooledSegment(BUFFER_SIZE), BUFFER_RECYCLER);
}
public static int fillBufferWithAscendingNumbers(Buffer buffer, int currentNumber) {
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java
index b0c702a..24d2864 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java
@@ -19,9 +19,11 @@
package org.apache.flink.runtime.io.disk.iomanager;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
import org.apache.flink.runtime.testutils.DiscardingRecycler;
+
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -194,7 +196,7 @@ public class BufferFileWriterReaderTest {
}
private Buffer createBuffer() {
- return new Buffer(new MemorySegment(new byte[BUFFER_SIZE]), BUFFER_RECYCLER);
+ return new Buffer(MemorySegmentFactory.allocateUnpooledSegment(BUFFER_SIZE), BUFFER_RECYCLER);
}
public static int fillBufferWithAscendingNumbers(Buffer buffer, int currentNumber) {
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java
index 435588f..4656d56 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java
@@ -19,6 +19,7 @@
package org.apache.flink.runtime.io.disk.iomanager;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -62,7 +63,7 @@ public class IOManagerAsyncTest {
final FileIOChannel.ID channelID = this.ioManager.createChannel();
final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channelID);
- MemorySegment memSeg = new MemorySegment(new byte[32 * 1024]);
+ MemorySegment memSeg = MemorySegmentFactory.allocateUnpooledSegment(32 * 1024);
for (int i = 0; i < NUM_IOS; i++) {
for (int pos = 0; pos < memSeg.size(); pos += 4) {
@@ -103,7 +104,7 @@ public class IOManagerAsyncTest {
try {
final List<MemorySegment> memSegs = new ArrayList<MemorySegment>();
for (int i = 0; i < NUM_SEGS; i++) {
- memSegs.add(new MemorySegment(new byte[32 * 1024]));
+ memSegs.add(MemorySegmentFactory.allocateUnpooledSegment(32 * 1024));
}
final FileIOChannel.ID channelID = this.ioManager.createChannel();
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
index 52908d3..6c25117 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
@@ -35,7 +35,7 @@ import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
/**
* Integration test case for the I/O manager.
@@ -54,11 +54,11 @@ public class IOManagerITCase {
private IOManager ioManager;
- private DefaultMemoryManager memoryManager;
+ private MemoryManager memoryManager;
@Before
public void beforeTest() {
- memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+ memoryManager = new MemoryManager(MEMORY_SIZE, 1);
ioManager = new IOManagerAsync();
}
@@ -209,7 +209,7 @@ public class IOManagerITCase {
}
}
- private static final int skewedSample(Random rnd, int max) {
+ private static int skewedSample(Random rnd, int max) {
double uniform = rnd.nextDouble();
double var = Math.pow(uniform, 8.0);
double pareto = 0.2 / var;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
index 3bdc9bd..fd02623 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
@@ -42,7 +42,7 @@ import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.junit.After;
import org.junit.Before;
@@ -66,14 +66,14 @@ public class IOManagerPerformanceBenchmark {
private static final AbstractInvokable memoryOwner = new DummyInvokable();
- private DefaultMemoryManager memManager;
+ private MemoryManager memManager;
private IOManager ioManager;
@Before
public void startup() {
- memManager = new DefaultMemoryManager(MEMORY_SIZE,1);
+ memManager = new MemoryManager(MEMORY_SIZE, 1);
ioManager = new IOManagerAsync();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
index d657ebf..420199c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.io.network;
import static org.junit.Assert.*;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.instance.DummyActorGateway;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.network.buffer.BufferPool;
@@ -55,7 +56,8 @@ public class NetworkEnvironmentTest {
try {
NettyConfig nettyConf = new NettyConfig(InetAddress.getLocalHost(), port, BUFFER_SIZE, new Configuration());
NetworkEnvironmentConfiguration config = new NetworkEnvironmentConfiguration(
- NUM_BUFFERS, BUFFER_SIZE, IOManager.IOMode.SYNC, new Some<NettyConfig>(nettyConf),
+ NUM_BUFFERS, BUFFER_SIZE, MemoryType.HEAP,
+ IOManager.IOMode.SYNC, new Some<NettyConfig>(nettyConf),
new Tuple2<Integer, Integer>(0, 0));
NetworkEnvironment env = new NetworkEnvironment(
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/PagedViewsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/PagedViewsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/PagedViewsTest.java
index e7d0524..8455402 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/PagedViewsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/PagedViewsTest.java
@@ -16,20 +16,23 @@
* limitations under the License.
*/
-
package org.apache.flink.runtime.io.network.api.serialization;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.network.api.serialization.types.SerializationTestType;
import org.apache.flink.runtime.io.network.api.serialization.types.SerializationTestTypeFactory;
import org.apache.flink.runtime.io.network.api.serialization.types.Util;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
+
import org.junit.Test;
import java.io.EOFException;
import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
@@ -375,7 +378,7 @@ public class PagedViewsTest {
private final int segmentSize;
private TestOutputView(int segmentSize) {
- super(new MemorySegment(new byte[segmentSize]), segmentSize, 0);
+ super(MemorySegmentFactory.allocateUnpooledSegment(segmentSize), segmentSize, 0);
this.segmentSize = segmentSize;
}
@@ -383,7 +386,7 @@ public class PagedViewsTest {
@Override
protected MemorySegment nextSegment(MemorySegment current, int positionInCurrent) throws IOException {
segments.add(new SegmentWithPosition(current, positionInCurrent));
- return new MemorySegment(new byte[segmentSize]);
+ return MemorySegmentFactory.allocateUnpooledSegment(segmentSize);
}
public void close() {
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java
index cd6d580..819a94f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java
@@ -19,11 +19,13 @@
package org.apache.flink.runtime.io.network.api.serialization;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.network.api.serialization.types.SerializationTestType;
import org.apache.flink.runtime.io.network.api.serialization.types.SerializationTestTypeFactory;
import org.apache.flink.runtime.io.network.api.serialization.types.Util;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+
import org.junit.Assert;
import org.junit.Test;
@@ -125,7 +127,7 @@ public class SpanningRecordSerializationTest {
{
final int SERIALIZATION_OVERHEAD = 4; // length encoding
- final Buffer buffer = new Buffer(new MemorySegment(new byte[segmentSize]), mock(BufferRecycler.class));
+ final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(segmentSize), mock(BufferRecycler.class));
final ArrayDeque<SerializationTestType> serializedRecords = new ArrayDeque<SerializationTestType>();
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializerTest.java
index 50d3639..b7bcb3e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializerTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.io.network.api.serialization;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.network.api.serialization.types.SerializationTestType;
import org.apache.flink.runtime.io.network.api.serialization.types.SerializationTestTypeFactory;
import org.apache.flink.runtime.io.network.api.serialization.types.Util;
@@ -41,7 +42,7 @@ public class SpanningRecordSerializerTest {
final int SEGMENT_SIZE = 16;
final SpanningRecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
- final Buffer buffer = new Buffer(new MemorySegment(new byte[SEGMENT_SIZE]), mock(BufferRecycler.class));
+ final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(SEGMENT_SIZE), mock(BufferRecycler.class));
final SerializationTestType randomIntRecord = Util.randomRecord(SerializationTestTypeFactory.INT);
Assert.assertFalse(serializer.hasData());
@@ -75,7 +76,7 @@ public class SpanningRecordSerializerTest {
final int SEGMENT_SIZE = 11;
final SpanningRecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
- final Buffer buffer = new Buffer(new MemorySegment(new byte[SEGMENT_SIZE]), mock(BufferRecycler.class));
+ final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(SEGMENT_SIZE), mock(BufferRecycler.class));
try {
Assert.assertEquals(RecordSerializer.SerializationResult.FULL_RECORD, serializer.setNextBuffer(buffer));
@@ -201,7 +202,7 @@ public class SpanningRecordSerializerTest {
final int SERIALIZATION_OVERHEAD = 4; // length encoding
final SpanningRecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
- final Buffer buffer = new Buffer(new MemorySegment(new byte[segmentSize]), mock(BufferRecycler.class));
+ final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(segmentSize), mock(BufferRecycler.class));
// -------------------------------------------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java
index 9e10582..f8cd28f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java
@@ -18,6 +18,7 @@
package org.apache.flink.runtime.io.network.api.writer;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.buffer.BufferPool;
import org.apache.flink.runtime.io.network.buffer.BufferProvider;
@@ -25,9 +26,11 @@ import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
import org.apache.flink.runtime.io.network.util.TestBufferFactory;
import org.apache.flink.runtime.io.network.util.TestTaskEvent;
import org.apache.flink.types.IntValue;
+
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
+
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.powermock.core.classloader.annotations.PrepareForTest;
@@ -41,14 +44,12 @@ import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import static com.google.common.base.Preconditions.checkNotNull;
+
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.spy;
@@ -158,7 +159,7 @@ public class RecordWriterTest {
BufferPool bufferPool = null;
try {
- buffers = new NetworkBufferPool(1, 1024);
+ buffers = new NetworkBufferPool(1, 1024, MemoryType.HEAP);
bufferPool = spy(buffers.createBufferPool(1, true));
ResultPartitionWriter partitionWriter = mock(ResultPartitionWriter.class);
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java
index 28862e8..0ac84dc 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java
@@ -18,6 +18,7 @@
package org.apache.flink.runtime.io.network.buffer;
+import org.apache.flink.core.memory.MemoryType;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -40,7 +41,7 @@ public class BufferPoolFactoryTest {
@Before
public void setupNetworkBufferPool() {
- networkBufferPool = new NetworkBufferPool(numBuffers, memorySegmentSize);
+ networkBufferPool = new NetworkBufferPool(numBuffers, memorySegmentSize, MemoryType.HEAP);
}
@After
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java
index 734dcfb..fd11d02 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java
@@ -19,6 +19,7 @@
package org.apache.flink.runtime.io.network.buffer;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
@@ -32,7 +33,7 @@ public class BufferTest {
@Test
public void testSetGetSize() {
- final MemorySegment segment = new MemorySegment(new byte[1024]);
+ final MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(1024);
final BufferRecycler recycler = Mockito.mock(BufferRecycler.class);
Buffer buffer = new Buffer(segment, recycler);
@@ -58,7 +59,7 @@ public class BufferTest {
@Test
public void testgetNioBufferThreadSafe() {
- final MemorySegment segment = new MemorySegment(new byte[1024]);
+ final MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(1024);
final BufferRecycler recycler = Mockito.mock(BufferRecycler.class);
Buffer buffer = new Buffer(segment, recycler);
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java
index e8e9ec8..93731e1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java
@@ -19,6 +19,7 @@
package org.apache.flink.runtime.io.network.buffer;
import com.google.common.collect.Lists;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.util.event.EventListener;
import org.junit.After;
import org.junit.AfterClass;
@@ -61,7 +62,7 @@ public class LocalBufferPoolTest {
@Before
public void setupLocalBufferPool() {
- networkBufferPool = new NetworkBufferPool(numBuffers, memorySegmentSize);
+ networkBufferPool = new NetworkBufferPool(numBuffers, memorySegmentSize, MemoryType.HEAP);
localBufferPool = new LocalBufferPool(networkBufferPool, 1);
assertEquals(0, localBufferPool.getNumberOfAvailableMemorySegments());
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java
index 6b22cd9..fd5c7a5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java
@@ -18,6 +18,7 @@
package org.apache.flink.runtime.io.network.buffer;
+import org.apache.flink.core.memory.MemoryType;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
@@ -35,7 +36,7 @@ public class NetworkBufferPoolTest {
final int bufferSize = 128;
final int numBuffers = 10;
- NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, bufferSize);
+ NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, bufferSize, MemoryType.HEAP);
assertEquals(bufferSize, globalPool.getMemorySegmentSize());
assertEquals(numBuffers, globalPool.getTotalNumberOfMemorySegments());
assertEquals(numBuffers, globalPool.getNumberOfAvailableMemorySegments());
@@ -70,7 +71,7 @@ public class NetworkBufferPoolTest {
@Test
public void testDestroyAll() {
try {
- NetworkBufferPool globalPool = new NetworkBufferPool(10, 128);
+ NetworkBufferPool globalPool = new NetworkBufferPool(10, 128, MemoryType.HEAP);
BufferPool fixedPool = globalPool.createBufferPool(2, true);
BufferPool nonFixedPool = globalPool.createBufferPool(5, false);
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
index 60241e3..f514cbd 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.io.network.netty;
import io.netty.buffer.ByteBuf;
import io.netty.channel.embedded.EmbeddedChannel;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.event.task.IntegerTaskEvent;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.io.network.buffer.Buffer;
@@ -53,7 +53,7 @@ public class NettyMessageSerializationTest {
@Test
public void testEncodeDecode() {
{
- Buffer buffer = spy(new Buffer(new MemorySegment(new byte[1024]), mock(BufferRecycler.class)));
+ Buffer buffer = spy(new Buffer(MemorySegmentFactory.allocateUnpooledSegment(1024), mock(BufferRecycler.class)));
ByteBuffer nioBuffer = buffer.getNioBuffer();
for (int i = 0; i < 1024; i += 4) {
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java
index 65780b4..cfbe99e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java
@@ -19,7 +19,7 @@
package org.apache.flink.runtime.io.network.partition.consumer;
import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer;
@@ -68,7 +68,7 @@ public class IteratorWrappingTestSingleInputGate<T extends IOReadableWritable> e
@Override
public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
if (inputIterator.next(reuse) != null) {
- final Buffer buffer = new Buffer(new MemorySegment(new byte[bufferSize]), mock(BufferRecycler.class));
+ final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(bufferSize), mock(BufferRecycler.class));
serializer.setNextBuffer(buffer);
serializer.addRecord(reuse);
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
index cd56318..ea40a55 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.io.network.partition.consumer;
import com.google.common.collect.Lists;
import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.execution.CancelTaskException;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
@@ -94,7 +95,7 @@ public class LocalInputChannelTest {
final NetworkBufferPool networkBuffers = new NetworkBufferPool(
(parallelism * producerBufferPoolSize) + (parallelism * parallelism),
- TestBufferFactory.BUFFER_SIZE);
+ TestBufferFactory.BUFFER_SIZE, MemoryType.HEAP);
final ResultPartitionConsumableNotifier partitionConsumableNotifier =
mock(ResultPartitionConsumableNotifier.class);
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
index 82cc730..f4c37f4 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
@@ -19,7 +19,7 @@
package org.apache.flink.runtime.io.network.partition.consumer;
import org.apache.flink.api.common.JobID;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
import org.apache.flink.runtime.deployment.ResultPartitionLocation;
import org.apache.flink.runtime.event.TaskEvent;
@@ -100,7 +100,8 @@ public class SingleInputGateTest {
when(taskEventDispatcher.publish(any(ResultPartitionID.class), any(TaskEvent.class))).thenReturn(true);
final ResultSubpartitionView iterator = mock(ResultSubpartitionView.class);
- when(iterator.getNextBuffer()).thenReturn(new Buffer(new MemorySegment(new byte[1024]), mock(BufferRecycler.class)));
+ when(iterator.getNextBuffer()).thenReturn(
+ new Buffer(MemorySegmentFactory.allocateUnpooledSegment(1024), mock(BufferRecycler.class)));
final ResultPartitionManager partitionManager = mock(ResultPartitionManager.class);
when(partitionManager.createSubpartitionView(any(ResultPartitionID.class), anyInt(), any(BufferProvider.class))).thenReturn(iterator);
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java
index d9e3562..d628596 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java
@@ -18,7 +18,7 @@
package org.apache.flink.runtime.io.network.serialization;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer;
import org.apache.flink.runtime.io.network.api.serialization.AdaptiveSpanningRecordDeserializer;
import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
@@ -52,7 +52,7 @@ public class LargeRecordsTest {
final RecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
final RecordDeserializer<SerializationTestType> deserializer = new AdaptiveSpanningRecordDeserializer<SerializationTestType>();
- final Buffer buffer = new Buffer(new MemorySegment(new byte[SEGMENT_SIZE]), mock(BufferRecycler.class));
+ final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(SEGMENT_SIZE), mock(BufferRecycler.class));
List<SerializationTestType> originalRecords = new ArrayList<SerializationTestType>();
List<SerializationTestType> deserializedRecords = new ArrayList<SerializationTestType>();
@@ -149,7 +149,7 @@ public class LargeRecordsTest {
final RecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
final RecordDeserializer<SerializationTestType> deserializer = new SpillingAdaptiveSpanningRecordDeserializer<SerializationTestType>();
- final Buffer buffer = new Buffer(new MemorySegment(new byte[SEGMENT_SIZE]), mock(BufferRecycler.class));
+ final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(SEGMENT_SIZE), mock(BufferRecycler.class));
List<SerializationTestType> originalRecords = new ArrayList<SerializationTestType>();
List<SerializationTestType> deserializedRecords = new ArrayList<SerializationTestType>();
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestBufferFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestBufferFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestBufferFactory.java
index cdba545..4b3b465 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestBufferFactory.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestBufferFactory.java
@@ -18,7 +18,9 @@
package org.apache.flink.runtime.io.network.util;
+import org.apache.flink.core.memory.HeapMemorySegment;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
import org.apache.flink.runtime.testutils.DiscardingRecycler;
@@ -59,7 +61,7 @@ public class TestBufferFactory {
public Buffer create() {
numberOfCreatedBuffers.incrementAndGet();
- return new Buffer(new MemorySegment(new byte[bufferSize]), bufferRecycler);
+ return new Buffer(MemorySegmentFactory.allocateUnpooledSegment(bufferSize), bufferRecycler);
}
public Buffer createFrom(MemorySegment segment) {
@@ -85,7 +87,7 @@ public class TestBufferFactory {
public static Buffer createBuffer(int bufferSize) {
checkArgument(bufferSize > 0);
- return new Buffer(new MemorySegment(new byte[bufferSize]), RECYCLER);
+ return new Buffer(MemorySegmentFactory.allocateUnpooledSegment(bufferSize), RECYCLER);
}
public static Buffer getMockBuffer() {
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java
index fc88207..15251e9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java
@@ -19,9 +19,8 @@
package org.apache.flink.runtime.memory;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.junit.After;
@@ -48,14 +47,14 @@ public class MemoryManagerLazyAllocationTest {
private static final int NUM_PAGES = MEMORY_SIZE / PAGE_SIZE;
- private DefaultMemoryManager memoryManager;
+ private MemoryManager memoryManager;
private Random random;
@Before
public void setUp() {
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, false);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, false);
this.random = new Random(RANDOM_SEED);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java
index c0f32ca..a20a180 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java
@@ -23,9 +23,8 @@ import java.util.List;
import java.util.Random;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.junit.Assert;
@@ -48,14 +47,14 @@ public class MemoryManagerTest {
private static final int NUM_PAGES = MEMORY_SIZE / PAGE_SIZE;
- private DefaultMemoryManager memoryManager;
+ private MemoryManager memoryManager;
private Random random;
@Before
public void setUp() {
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, true);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
this.random = new Random(RANDOM_SEED);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSimpleTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSimpleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSimpleTest.java
new file mode 100644
index 0000000..fad1b0e
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSimpleTest.java
@@ -0,0 +1,576 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.memory;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.nio.ByteBuffer;
+import java.util.Random;
+
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.core.memory.MemoryType;
+import org.apache.flink.runtime.operators.testutils.DummyInvokable;
+import org.apache.flink.core.memory.MemorySegment;
+
+import org.junit.Assert;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class MemorySegmentSimpleTest {
+
+ public static final long RANDOM_SEED = 643196033469871L;
+
+ public static final int MANAGED_MEMORY_SIZE = 1024 * 1024 * 16;
+
+ public static final int PAGE_SIZE = 1024 * 512;
+
+ private MemoryManager manager;
+
+ private MemorySegment segment;
+
+ private Random random;
+
+ @Before
+ public void setUp() throws Exception{
+ try {
+ this.manager = new MemoryManager(MANAGED_MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
+ this.segment = manager.allocatePages(new DummyInvokable(), 1).get(0);
+ this.random = new Random(RANDOM_SEED);
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail("Test setup failed.");
+ }
+ }
+
+ @After
+ public void tearDown() {
+ this.manager.release(this.segment);
+ this.random = null;
+ this.segment = null;
+
+ if (!this.manager.verifyEmpty()) {
+ Assert.fail("Not all memory has been properly released.");
+ }
+ this.manager = null;
+ }
+
+ @Test
+ public void bulkByteAccess() {
+
+ // test exceptions
+ {
+ byte[] bytes = new byte[PAGE_SIZE / 4];
+
+ try {
+ segment.put(3 * (PAGE_SIZE / 4) + 1, bytes);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(7 * (PAGE_SIZE / 8) + 1, bytes, 0, bytes.length / 2);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ }
+
+ // test expected correct behavior with default offset / length
+ {
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ byte[] src = new byte[PAGE_SIZE / 8];
+ for (int i = 0; i < 8; i++) {
+ random.nextBytes(src);
+ segment.put(i * (PAGE_SIZE / 8), src);
+ }
+
+ random.setSeed(seed);
+ byte[] expected = new byte[PAGE_SIZE / 8];
+ byte[] actual = new byte[PAGE_SIZE / 8];
+ for (int i = 0; i < 8; i++) {
+ random.nextBytes(expected);
+ segment.get(i * (PAGE_SIZE / 8), actual);
+
+ assertArrayEquals(expected, actual);
+ }
+ }
+
+ // test expected correct behavior with specific offset / length
+ {
+ byte[] expected = new byte[PAGE_SIZE];
+ random.nextBytes(expected);
+
+ for (int i = 0; i < 16; i++) {
+ segment.put(i * (PAGE_SIZE / 16), expected, i * (PAGE_SIZE / 16),
+ PAGE_SIZE / 16);
+ }
+
+ byte[] actual = new byte[PAGE_SIZE];
+ for (int i = 0; i < 16; i++) {
+ segment.get(i * (PAGE_SIZE / 16), actual, i * (PAGE_SIZE / 16),
+ PAGE_SIZE / 16);
+ }
+
+ assertArrayEquals(expected, actual);
+ }
+ }
+
+ @Test
+ public void byteAccess() {
+ // test exceptions
+ {
+ try {
+ segment.put(-1, (byte) 0);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(PAGE_SIZE, (byte) 0);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(-1);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(PAGE_SIZE);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ }
+
+ // test expected correct behavior
+ {
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i < PAGE_SIZE; i++) {
+ segment.put(i, (byte) random.nextInt());
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i < PAGE_SIZE; i++) {
+ assertEquals((byte) random.nextInt(), segment.get(i));
+ }
+ }
+ }
+
+ @Test
+ public void booleanAccess() {
+ // test exceptions
+ {
+ try {
+ segment.putBoolean(-1, false);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putBoolean(PAGE_SIZE, false);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getBoolean(-1);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getBoolean(PAGE_SIZE);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ }
+
+ // test expected correct behavior
+ {
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i < PAGE_SIZE; i++) {
+ segment.putBoolean(i, random.nextBoolean());
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i < PAGE_SIZE; i++) {
+ assertEquals(random.nextBoolean(), segment.getBoolean(i));
+ }
+ }
+ }
+
+ @Test
+ public void charAccess() {
+ // test exceptions
+ {
+ try {
+ segment.putChar(-1, 'a');
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putChar(PAGE_SIZE, 'a');
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getChar(-1);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getChar(PAGE_SIZE);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ }
+
+ // test expected correct behavior
+ {
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
+ segment.putChar(i, (char) ('a' + random.nextInt(26)));
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
+ assertEquals((char) ('a' + random.nextInt(26)), segment.getChar(i));
+ }
+ }
+ }
+
+ @Test
+ public void doubleAccess() {
+ // test exceptions
+ {
+ try {
+ segment.putDouble(-1, 0.0);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putDouble(PAGE_SIZE, 0.0);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getDouble(-1);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getDouble(PAGE_SIZE);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ }
+
+ // test expected correct behavior
+ {
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
+ segment.putDouble(i, random.nextDouble());
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
+ assertEquals(random.nextDouble(), segment.getDouble(i), 0.0);
+ }
+ }
+ }
+
+ // @Test
+ public void floatAccess() {
+ // test exceptions
+ {
+ try {
+ segment.putFloat(-1, 0.0f);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putFloat(PAGE_SIZE, 0.0f);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getFloat(-1);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getFloat(PAGE_SIZE);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ }
+
+ // test expected correct behavior
+ {
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
+ segment.putFloat(i, random.nextFloat());
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
+ assertEquals(random.nextFloat(), segment.getFloat(i), 0.0);
+ }
+ }
+ }
+
+ @Test
+ public void longAccess() {
+ // test exceptions
+ {
+ try {
+ segment.putLong(-1, 0L);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putLong(PAGE_SIZE, 0L);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getLong(-1);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getLong(PAGE_SIZE);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ }
+
+ // test expected correct behavior
+ {
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
+ segment.putLong(i, random.nextLong());
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
+ assertEquals(random.nextLong(), segment.getLong(i));
+ }
+ }
+
+ // test unaligned offsets
+ {
+ final long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int offset = 0; offset < PAGE_SIZE - 8; offset += random.nextInt(24) + 8) {
+ long value = random.nextLong();
+ segment.putLong(offset, value);
+ }
+
+ random.setSeed(seed);
+ for (int offset = 0; offset < PAGE_SIZE - 8; offset += random.nextInt(24) + 8) {
+ long shouldValue = random.nextLong();
+ long isValue = segment.getLong(offset);
+ assertEquals(shouldValue, isValue);
+ }
+ }
+ }
+
+ @Test
+ public void intAccess() {
+ // test exceptions
+ {
+ try {
+ segment.putInt(-1, 0);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putInt(PAGE_SIZE, 0);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getInt(-1);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getInt(PAGE_SIZE);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ }
+
+ // test expected correct behavior
+ {
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
+ segment.putInt(i, random.nextInt());
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
+ assertEquals(random.nextInt(), segment.getInt(i));
+ }
+ }
+ }
+
+ @Test
+ public void shortAccess() {
+ // test exceptions
+ {
+ try {
+ segment.putShort(-1, (short) 0);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putShort(PAGE_SIZE, (short) 0);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getShort(-1);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getShort(PAGE_SIZE);
+ fail("IndexOutOfBoundsException expected");
+ } catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ }
+
+ // test expected correct behavior
+ {
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
+ segment.putShort(i, (short) random.nextInt());
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
+ assertEquals((short) random.nextInt(), segment.getShort(i));
+ }
+ }
+ }
+
+ @Test
+ public void testByteBufferWrapping() {
+ try {
+ MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(1024);
+
+ ByteBuffer buf1 = seg.wrap(13, 47);
+ assertEquals(13, buf1.position());
+ assertEquals(60, buf1.limit());
+ assertEquals(47, buf1.remaining());
+
+ ByteBuffer buf2 = seg.wrap(500, 267);
+ assertEquals(500, buf2.position());
+ assertEquals(767, buf2.limit());
+ assertEquals(267, buf2.remaining());
+
+ ByteBuffer buf3 = seg.wrap(0, 1024);
+ assertEquals(0, buf3.position());
+ assertEquals(1024, buf3.limit());
+ assertEquals(1024, buf3.remaining());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+}
[05/15] flink git commit: [FLINK-1320] [core] Add an off-heap variant
of the managed memory
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSpeedBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSpeedBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSpeedBenchmark.java
deleted file mode 100644
index 0843cc2..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSpeedBenchmark.java
+++ /dev/null
@@ -1,867 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.memory;
-
-import java.util.Arrays;
-
-import org.apache.flink.runtime.memorymanager.CheckedMemorySegment;
-import org.apache.flink.runtime.memorymanager.SimpleMemorySegment;
-import org.apache.flink.runtime.memorymanager.UnsafeMemorySegment;
-
-/**
- *
- */
-public class MemorySegmentSpeedBenchmark {
-
- private static final long LONG_VALUE = 0x1234567890abcdefl;
-
- private static final int INT_VALUE = 0x12345678;
-
- private static final byte BYTE_VALUE = 0x56;
-
- @SuppressWarnings("unused")
- private static long sideEffect = 0;
-
-
- public static void main(String[] args) {
- final int SMALL_SEGMENT_SIZE = 32 * 1024;
- final int LARGE_SEGMENT_SIZE = 1024 * 1024 * 1024;
-
- final int SMALL_SEGMENTS_ROUNDS = 50000;
- final int LARGE_SEGMENT_ROUNDS = 10;
-
- final byte[] largeSegment = new byte[LARGE_SEGMENT_SIZE];
- final byte[] smallSegment = new byte[SMALL_SEGMENT_SIZE];
-
- testPutLongs(smallSegment, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
- testGetLongs(smallSegment, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
- testPutLongs(largeSegment, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
- testGetLongs(largeSegment, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
-
- testPutLongsBigEndian(smallSegment, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
- testGetLongsBigEndian(smallSegment, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
- testPutLongsBigEndian(largeSegment, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
- testGetLongsBigEndian(largeSegment, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
-
- testPutLongsLittleEndian(smallSegment, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
- testGetLongsLittleEndian(smallSegment, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
- testPutLongsLittleEndian(largeSegment, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
- testGetLongsLittleEndian(largeSegment, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
-
- testPutInts(smallSegment, SMALL_SEGMENT_SIZE / 4, SMALL_SEGMENTS_ROUNDS);
- testGetInts(smallSegment, SMALL_SEGMENT_SIZE / 4, SMALL_SEGMENTS_ROUNDS);
- testPutInts(largeSegment, LARGE_SEGMENT_SIZE / 4, LARGE_SEGMENT_ROUNDS);
- testGetInts(largeSegment, LARGE_SEGMENT_SIZE / 4, LARGE_SEGMENT_ROUNDS);
-
- testPutBytes(smallSegment, SMALL_SEGMENT_SIZE, SMALL_SEGMENTS_ROUNDS);
- testGetBytes(smallSegment, SMALL_SEGMENT_SIZE, SMALL_SEGMENTS_ROUNDS);
- testPutBytes(largeSegment, LARGE_SEGMENT_SIZE, LARGE_SEGMENT_ROUNDS);
- testGetBytes(largeSegment, LARGE_SEGMENT_SIZE, LARGE_SEGMENT_ROUNDS);
-
- testPutByteArrays1024(smallSegment, SMALL_SEGMENT_SIZE / 1024, SMALL_SEGMENTS_ROUNDS);
- testGetByteArrays1024(smallSegment, SMALL_SEGMENT_SIZE / 1024, SMALL_SEGMENTS_ROUNDS);
- testPutByteArrays1024(largeSegment, LARGE_SEGMENT_SIZE / 1024, LARGE_SEGMENT_ROUNDS);
- testGetByteArrays1024(largeSegment, LARGE_SEGMENT_SIZE / 1024, LARGE_SEGMENT_ROUNDS);
- }
-
- // --------------------------------------------------------------------------------------------
- // LONGs
- // --------------------------------------------------------------------------------------------
-
- private static final void testPutLongs(byte[] segmentArray, int numValues, int rounds) {
- Arrays.fill(segmentArray, (byte) 0);
- CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
- long elapsedChecked = timePutLongsChecked(checkedSegment, numValues, rounds);
- checkedSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
- long elapsedDirect = timePutLongsDirect(directSegment, numValues, rounds);
- directSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
- long elapsedUnsafe = timePutLongsUnsafe(unsafeSegment, numValues, rounds);
- unsafeSegment = null;
-
- System.out.println(String.format("Writing %d x %d longs to %d segment: checked=%,d nsecs, direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
- }
-
- private static final void testGetLongs(byte[] segmentArray, int numValues, int rounds) {
- Arrays.fill(segmentArray, (byte) 0);
- CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
- long elapsedChecked = timeGetLongsChecked(checkedSegment, numValues, rounds);
- checkedSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
- long elapsedDirect = timeGetLongsDirect(directSegment, numValues, rounds);
- directSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
- long elapsedUnsafe = timeGetLongsUnsafe(unsafeSegment, numValues, rounds);
- unsafeSegment = null;
-
- System.out.println(String.format("Reading %d x %d longs from %d segment: checked=%,d nsecs, direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
- }
-
- private static long timePutLongsChecked(final CheckedMemorySegment checked, final int num, final int rounds) {
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- checked.putLong(offset, LONG_VALUE);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timePutLongsDirect(final SimpleMemorySegment direct, final int num, final int rounds) {
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- direct.putLong(offset, LONG_VALUE);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timePutLongsUnsafe(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- unsafe.putLong(offset, LONG_VALUE);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timeGetLongsChecked(final CheckedMemorySegment checked, final int num, final int rounds) {
- long l = 0;
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- l += checked.getLong(offset);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- sideEffect += l;
- return end - start;
- }
-
- private static long timeGetLongsDirect(final SimpleMemorySegment direct, final int num, final int rounds) {
- long l = 0;
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- l += direct.getLong(offset);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- sideEffect += l;
- return end - start;
- }
-
- private static long timeGetLongsUnsafe(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
- // checked segment
- long l = 0;
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- l += unsafe.getLong(offset);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- sideEffect += l;
- return end - start;
- }
-
- // --------------------------------------------------------------------------------------------
- // LONG BIG ENDIAN
- // --------------------------------------------------------------------------------------------
-
- private static final void testPutLongsBigEndian(byte[] segmentArray, int numValues, int rounds) {
- Arrays.fill(segmentArray, (byte) 0);
- CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
- long elapsedChecked = timePutLongsCheckedBigEndian(checkedSegment, numValues, rounds);
- checkedSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
- long elapsedDirect = timePutLongsDirectBigEndian(directSegment, numValues, rounds);
- directSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
- long elapsedUnsafe = timePutLongsUnsafeBigEndian(unsafeSegment, numValues, rounds);
- unsafeSegment = null;
-
- System.out.println(String.format("Writing %d x %d big endian longs to %d segment: checked=%,d nsecs, direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
- }
-
- private static final void testGetLongsBigEndian(byte[] segmentArray, int numValues, int rounds) {
- Arrays.fill(segmentArray, (byte) 0);
- CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
- long elapsedChecked = timeGetLongsCheckedBigEndian(checkedSegment, numValues, rounds);
- checkedSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
- long elapsedDirect = timeGetLongsDirectBigEndian(directSegment, numValues, rounds);
- directSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
- long elapsedUnsafe = timeGetLongsUnsafeBigEndian(unsafeSegment, numValues, rounds);
- unsafeSegment = null;
-
- System.out.println(String.format("Reading %d x %d big endian longs from %d segment: checked=%,d nsecs, direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
- }
-
- private static long timePutLongsCheckedBigEndian(final CheckedMemorySegment checked, final int num, final int rounds) {
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- checked.putLongBigEndian(offset, LONG_VALUE);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timePutLongsDirectBigEndian(final SimpleMemorySegment direct, final int num, final int rounds) {
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- direct.putLongBigEndian(offset, LONG_VALUE);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timePutLongsUnsafeBigEndian(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- unsafe.putLongBigEndian(offset, LONG_VALUE);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timeGetLongsCheckedBigEndian(final CheckedMemorySegment checked, final int num, final int rounds) {
- long l = 0;
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- l += checked.getLongBigEndian(offset);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- sideEffect += l;
- return end - start;
- }
-
- private static long timeGetLongsDirectBigEndian(final SimpleMemorySegment direct, final int num, final int rounds) {
- long l = 0;
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- l += direct.getLongBigEndian(offset);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- sideEffect += l;
- return end - start;
- }
-
- private static long timeGetLongsUnsafeBigEndian(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
- // checked segment
- long l = 0;
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- l += unsafe.getLongBigEndian(offset);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- sideEffect += l;
- return end - start;
- }
-
- // --------------------------------------------------------------------------------------------
- // LONG LITTLE ENDIAN
- // --------------------------------------------------------------------------------------------
-
- private static final void testPutLongsLittleEndian(byte[] segmentArray, int numValues, int rounds) {
- Arrays.fill(segmentArray, (byte) 0);
- CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
- long elapsedChecked = timePutLongsCheckedLittleEndian(checkedSegment, numValues, rounds);
- checkedSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
- long elapsedDirect = timePutLongsDirectLittleEndian(directSegment, numValues, rounds);
- directSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
- long elapsedUnsafe = timePutLongsUnsafeLittleEndian(unsafeSegment, numValues, rounds);
- unsafeSegment = null;
-
- System.out.println(String.format("Writing %d x %d little endian longs to %d segment: checked=%,d nsecs, direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
- }
-
- private static final void testGetLongsLittleEndian(byte[] segmentArray, int numValues, int rounds) {
- Arrays.fill(segmentArray, (byte) 0);
- CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
- long elapsedChecked = timeGetLongsCheckedLittleEndian(checkedSegment, numValues, rounds);
- checkedSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
- long elapsedDirect = timeGetLongsDirectLittleEndian(directSegment, numValues, rounds);
- directSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
- long elapsedUnsafe = timeGetLongsUnsafeLittleEndian(unsafeSegment, numValues, rounds);
- unsafeSegment = null;
-
- System.out.println(String.format("Reading %d x %d little endian longs from %d segment: checked=%,d nsecs, direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
- }
-
- private static long timePutLongsCheckedLittleEndian(final CheckedMemorySegment checked, final int num, final int rounds) {
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- checked.putLongLittleEndian(offset, LONG_VALUE);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timePutLongsDirectLittleEndian(final SimpleMemorySegment direct, final int num, final int rounds) {
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- direct.putLongLittleEndian(offset, LONG_VALUE);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timePutLongsUnsafeLittleEndian(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- unsafe.putLongLittleEndian(offset, LONG_VALUE);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timeGetLongsCheckedLittleEndian(final CheckedMemorySegment checked, final int num, final int rounds) {
- long l = 0;
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- l += checked.getLongLittleEndian(offset);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- sideEffect += l;
- return end - start;
- }
-
- private static long timeGetLongsDirectLittleEndian(final SimpleMemorySegment direct, final int num, final int rounds) {
- long l = 0;
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- l += direct.getLongLittleEndian(offset);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- sideEffect += l;
- return end - start;
- }
-
- private static long timeGetLongsUnsafeLittleEndian(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
- // checked segment
- long l = 0;
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- l += unsafe.getLongLittleEndian(offset);
- offset += 8;
- }
- }
- long end = System.nanoTime();
- sideEffect += l;
- return end - start;
- }
-
- // --------------------------------------------------------------------------------------------
- // INTs
- // --------------------------------------------------------------------------------------------
-
- private static final void testPutInts(byte[] segmentArray, int numValues, int rounds) {
- Arrays.fill(segmentArray, (byte) 0);
- CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
- long elapsedChecked = timePutIntsChecked(checkedSegment, numValues, rounds);
- checkedSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
- long elapsedDirect = timePutIntsDirect(directSegment, numValues, rounds);
- directSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
- long elapsedUnsafe = timePutIntsUnsafe(unsafeSegment, numValues, rounds);
- unsafeSegment = null;
-
- System.out.println(String.format("Writing %d x %d ints to %d segment: checked=%,d nsecs, direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
- }
-
- private static final void testGetInts(byte[] segmentArray, int numValues, int rounds) {
- Arrays.fill(segmentArray, (byte) 0);
- CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
- long elapsedChecked = timeGetIntsChecked(checkedSegment, numValues, rounds);
- checkedSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
- long elapsedDirect = timeGetIntsDirect(directSegment, numValues, rounds);
- directSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
- long elapsedUnsafe = timeGetIntsUnsafe(unsafeSegment, numValues, rounds);
- unsafeSegment = null;
-
- System.out.println(String.format("Reading %d x %d ints from %d segment: checked=%,d nsecs, direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
- }
-
- private static long timePutIntsChecked(final CheckedMemorySegment checked, final int num, final int rounds) {
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- checked.putInt(offset, INT_VALUE);
- offset += 4;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timePutIntsDirect(final SimpleMemorySegment direct, final int num, final int rounds) {
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- direct.putInt(offset, INT_VALUE);
- offset += 4;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timePutIntsUnsafe(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- unsafe.putInt(offset, INT_VALUE);
- offset += 4;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timeGetIntsChecked(final CheckedMemorySegment checked, final int num, final int rounds) {
- int l = 0;
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- l += checked.getInt(offset);
- offset += 4;
- }
- }
- long end = System.nanoTime();
- sideEffect += l;
- return end - start;
- }
-
- private static long timeGetIntsDirect(final SimpleMemorySegment direct, final int num, final int rounds) {
- int l = 0;
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- l += direct.getInt(offset);
- offset += 4;
- }
- }
- long end = System.nanoTime();
- sideEffect += l;
- return end - start;
- }
-
- private static long timeGetIntsUnsafe(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
- int l = 0;
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- l += unsafe.getInt(offset);
- offset += 4;
- }
- }
- long end = System.nanoTime();
- sideEffect += l;
- return end - start;
- }
-
- // --------------------------------------------------------------------------------------------
- // BYTEs
- // --------------------------------------------------------------------------------------------
-
- private static final void testPutBytes(byte[] segmentArray, int numValues, int rounds) {
- Arrays.fill(segmentArray, (byte) 0);
- CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
- long elapsedChecked = timePutBytesChecked(checkedSegment, numValues, rounds);
- checkedSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
- long elapsedDirect = timePutBytesDirect(directSegment, numValues, rounds);
- directSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
- long elapsedUnsafe = timePutBytesUnsafe(unsafeSegment, numValues, rounds);
- unsafeSegment = null;
-
- System.out.println(String.format("Writing %d x %d bytes to %d segment: checked=%,d nsecs, direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
- }
-
- private static final void testGetBytes(byte[] segmentArray, int numValues, int rounds) {
- Arrays.fill(segmentArray, (byte) 0);
- CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
- long elapsedChecked = timeGetBytesChecked(checkedSegment, numValues, rounds);
- checkedSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
- long elapsedDirect = timeGetBytesDirect(directSegment, numValues, rounds);
- directSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
- long elapsedUnsafe = timeGetBytesUnsafe(unsafeSegment, numValues, rounds);
- unsafeSegment = null;
-
- System.out.println(String.format("Reading %d x %d bytes from %d segment: checked=%,d nsecs, direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
- }
-
- private static long timePutBytesChecked(final CheckedMemorySegment checked, final int num, final int rounds) {
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- checked.put(offset, BYTE_VALUE);
- offset++;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timePutBytesDirect(final SimpleMemorySegment direct, final int num, final int rounds) {
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- direct.put(offset, BYTE_VALUE);
- offset++;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timePutBytesUnsafe(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- unsafe.put(offset, BYTE_VALUE);
- offset++;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timeGetBytesChecked(final CheckedMemorySegment checked, final int num, final int rounds) {
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- checked.get(offset);
- offset++;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timeGetBytesDirect(final SimpleMemorySegment direct, final int num, final int rounds) {
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- direct.get(offset);
- offset++;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timeGetBytesUnsafe(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- unsafe.get(offset);
- offset++;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- // --------------------------------------------------------------------------------------------
- // BYTE ARRAYs
- // --------------------------------------------------------------------------------------------
-
- private static final void testPutByteArrays1024(byte[] segmentArray, int numValues, int rounds) {
- byte[] sourceArray = new byte[1024];
- for (int i = 0; i < sourceArray.length; i++) {
- sourceArray[i] = (byte) (i % Byte.MAX_VALUE);
- }
-
- Arrays.fill(segmentArray, (byte) 0);
- CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
- long elapsedChecked = timePutByteArrayChecked(checkedSegment, sourceArray, numValues, rounds);
- checkedSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
- long elapsedDirect = timePutByteArrayDirect(directSegment, sourceArray, numValues, rounds);
- directSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
- long elapsedUnsafe = timePutByteArrayUnsafe(unsafeSegment, sourceArray, numValues, rounds);
- unsafeSegment = null;
-
- System.out.println(String.format("Writing %d x %d byte[1024] to %d segment: checked=%,d nsecs, direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
- }
-
- private static final void testGetByteArrays1024(byte[] segmentArray, int numValues, int rounds) {
- byte[] targetArray = new byte[1024];
-
- Arrays.fill(segmentArray, (byte) 0);
- CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
- long elapsedChecked = timeGetByteArrayChecked(checkedSegment, targetArray, numValues, rounds);
- checkedSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
- long elapsedDirect = timeGetByteArrayDirect(directSegment, targetArray, numValues, rounds);
- directSegment = null;
-
- Arrays.fill(segmentArray, (byte) 0);
- UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
- long elapsedUnsafe = timeGetByteArrayUnsafe(unsafeSegment, targetArray, numValues, rounds);
- unsafeSegment = null;
-
- System.out.println(String.format("Reading %d x %d byte[1024] from %d segment: checked=%,d nsecs, direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
- }
-
-
- private static long timePutByteArrayChecked(final CheckedMemorySegment checked, final byte[] source, final int num, final int rounds) {
- final int len = source.length;
-
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- checked.put(offset, source, 0, len);
- offset += len;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timePutByteArrayDirect(final SimpleMemorySegment direct, final byte[] source, final int num, final int rounds) {
- final int len = source.length;
-
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- direct.put(offset, source, 0, len);
- offset += len;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timePutByteArrayUnsafe(final UnsafeMemorySegment unsafe, final byte[] source, final int num, final int rounds) {
- final int len = source.length;
-
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- unsafe.put(offset, source, 0, len);
- offset += len;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timeGetByteArrayChecked(final CheckedMemorySegment checked, final byte[] target, final int num, final int rounds) {
- final int len = target.length;
-
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- checked.get(offset, target, 0, len);
- offset += len;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timeGetByteArrayDirect(final SimpleMemorySegment direct, final byte[] target, final int num, final int rounds) {
- final int len = target.length;
-
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- direct.get(offset, target, 0, len);
- offset += len;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-
- private static long timeGetByteArrayUnsafe(final UnsafeMemorySegment unsafe, final byte[] target, final int num, final int rounds) {
- final int len = target.length;
-
- // checked segment
- long start = System.nanoTime();
- for (int round = 0; round < rounds; round++) {
- int offset = 0;
- for (int i = 0; i < num; i++) {
- unsafe.get(offset, target, 0, len);
- offset += len;
- }
- }
- long end = System.nanoTime();
- return end - start;
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java
deleted file mode 100644
index 990dffa..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java
+++ /dev/null
@@ -1,575 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.runtime.memory;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.nio.ByteBuffer;
-import java.util.Random;
-
-import org.apache.flink.runtime.operators.testutils.DummyInvokable;
-import org.junit.Assert;
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-public class MemorySegmentTest {
-
- public static final long RANDOM_SEED = 643196033469871L;
-
- public static final int MANAGED_MEMORY_SIZE = 1024 * 1024 * 16;
-
- public static final int PAGE_SIZE = 1024 * 512;
-
- private DefaultMemoryManager manager;
-
- private MemorySegment segment;
-
- private Random random;
-
- @Before
- public void setUp() throws Exception{
- try {
- this.manager = new DefaultMemoryManager(MANAGED_MEMORY_SIZE, 1, PAGE_SIZE, true);
- this.segment = manager.allocatePages(new DummyInvokable(), 1).get(0);
- this.random = new Random(RANDOM_SEED);
- } catch (Exception e) {
- e.printStackTrace();
- fail("Test setup failed.");
- }
- }
-
- @After
- public void tearDown() {
- this.manager.release(this.segment);
- this.random = null;
- this.segment = null;
-
- if (!this.manager.verifyEmpty()) {
- Assert.fail("Not all memory has been properly released.");
- }
- this.manager = null;
- }
-
- @Test
- public void bulkByteAccess() {
-
- // test exceptions
- {
- byte[] bytes = new byte[PAGE_SIZE / 4];
-
- try {
- segment.put(3 * (PAGE_SIZE / 4) + 1, bytes);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.put(7 * (PAGE_SIZE / 8) + 1, bytes, 0, bytes.length / 2);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
- }
-
- // test expected correct behavior with default offset / length
- {
- long seed = random.nextLong();
-
- random.setSeed(seed);
- byte[] src = new byte[PAGE_SIZE / 8];
- for (int i = 0; i < 8; i++) {
- random.nextBytes(src);
- segment.put(i * (PAGE_SIZE / 8), src);
- }
-
- random.setSeed(seed);
- byte[] expected = new byte[PAGE_SIZE / 8];
- byte[] actual = new byte[PAGE_SIZE / 8];
- for (int i = 0; i < 8; i++) {
- random.nextBytes(expected);
- segment.get(i * (PAGE_SIZE / 8), actual);
-
- assertArrayEquals(expected, actual);
- }
- }
-
- // test expected correct behavior with specific offset / length
- {
- byte[] expected = new byte[PAGE_SIZE];
- random.nextBytes(expected);
-
- for (int i = 0; i < 16; i++) {
- segment.put(i * (PAGE_SIZE / 16), expected, i * (PAGE_SIZE / 16),
- PAGE_SIZE / 16);
- }
-
- byte[] actual = new byte[PAGE_SIZE];
- for (int i = 0; i < 16; i++) {
- segment.get(i * (PAGE_SIZE / 16), actual, i * (PAGE_SIZE / 16),
- PAGE_SIZE / 16);
- }
-
- assertArrayEquals(expected, actual);
- }
- }
-
- @Test
- public void byteAccess() {
- // test exceptions
- {
- try {
- segment.put(-1, (byte) 0);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.put(PAGE_SIZE, (byte) 0);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.get(-1);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.get(PAGE_SIZE);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
- }
-
- // test expected correct behavior
- {
- long seed = random.nextLong();
-
- random.setSeed(seed);
- for (int i = 0; i < PAGE_SIZE; i++) {
- segment.put(i, (byte) random.nextInt());
- }
-
- random.setSeed(seed);
- for (int i = 0; i < PAGE_SIZE; i++) {
- assertEquals((byte) random.nextInt(), segment.get(i));
- }
- }
- }
-
- @Test
- public void booleanAccess() {
- // test exceptions
- {
- try {
- segment.putBoolean(-1, false);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.putBoolean(PAGE_SIZE, false);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.getBoolean(-1);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.getBoolean(PAGE_SIZE);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
- }
-
- // test expected correct behavior
- {
- long seed = random.nextLong();
-
- random.setSeed(seed);
- for (int i = 0; i < PAGE_SIZE; i++) {
- segment.putBoolean(i, random.nextBoolean());
- }
-
- random.setSeed(seed);
- for (int i = 0; i < PAGE_SIZE; i++) {
- assertEquals(random.nextBoolean(), segment.getBoolean(i));
- }
- }
- }
-
- @Test
- public void charAccess() {
- // test exceptions
- {
- try {
- segment.putChar(-1, 'a');
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.putChar(PAGE_SIZE, 'a');
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.getChar(-1);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.getChar(PAGE_SIZE);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
- }
-
- // test expected correct behavior
- {
- long seed = random.nextLong();
-
- random.setSeed(seed);
- for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
- segment.putChar(i, (char) ('a' + random.nextInt(26)));
- }
-
- random.setSeed(seed);
- for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
- assertEquals((char) ('a' + random.nextInt(26)), segment.getChar(i));
- }
- }
- }
-
- @Test
- public void doubleAccess() {
- // test exceptions
- {
- try {
- segment.putDouble(-1, 0.0);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.putDouble(PAGE_SIZE, 0.0);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.getDouble(-1);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.getDouble(PAGE_SIZE);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
- }
-
- // test expected correct behavior
- {
- long seed = random.nextLong();
-
- random.setSeed(seed);
- for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
- segment.putDouble(i, random.nextDouble());
- }
-
- random.setSeed(seed);
- for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
- assertEquals(random.nextDouble(), segment.getDouble(i), 0.0);
- }
- }
- }
-
- // @Test
- public void floatAccess() {
- // test exceptions
- {
- try {
- segment.putFloat(-1, 0.0f);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.putFloat(PAGE_SIZE, 0.0f);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.getFloat(-1);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.getFloat(PAGE_SIZE);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
- }
-
- // test expected correct behavior
- {
- long seed = random.nextLong();
-
- random.setSeed(seed);
- for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
- segment.putFloat(i, random.nextFloat());
- }
-
- random.setSeed(seed);
- for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
- assertEquals(random.nextFloat(), segment.getFloat(i), 0.0);
- }
- }
- }
-
- @Test
- public void longAccess() {
- // test exceptions
- {
- try {
- segment.putLong(-1, 0L);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.putLong(PAGE_SIZE, 0L);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.getLong(-1);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.getLong(PAGE_SIZE);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
- }
-
- // test expected correct behavior
- {
- long seed = random.nextLong();
-
- random.setSeed(seed);
- for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
- segment.putLong(i, random.nextLong());
- }
-
- random.setSeed(seed);
- for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
- assertEquals(random.nextLong(), segment.getLong(i));
- }
- }
-
- // test unaligned offsets
- {
- final long seed = random.nextLong();
-
- random.setSeed(seed);
- for (int offset = 0; offset < PAGE_SIZE - 8; offset += random.nextInt(24) + 8) {
- long value = random.nextLong();
- segment.putLong(offset, value);
- }
-
- random.setSeed(seed);
- for (int offset = 0; offset < PAGE_SIZE - 8; offset += random.nextInt(24) + 8) {
- long shouldValue = random.nextLong();
- long isValue = segment.getLong(offset);
- assertEquals(shouldValue, isValue);
- }
- }
- }
-
- @Test
- public void intAccess() {
- // test exceptions
- {
- try {
- segment.putInt(-1, 0);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.putInt(PAGE_SIZE, 0);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.getInt(-1);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.getInt(PAGE_SIZE);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
- }
-
- // test expected correct behavior
- {
- long seed = random.nextLong();
-
- random.setSeed(seed);
- for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
- segment.putInt(i, random.nextInt());
- }
-
- random.setSeed(seed);
- for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
- assertEquals(random.nextInt(), segment.getInt(i));
- }
- }
- }
-
- @Test
- public void shortAccess() {
- // test exceptions
- {
- try {
- segment.putShort(-1, (short) 0);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.putShort(PAGE_SIZE, (short) 0);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.getShort(-1);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
-
- try {
- segment.getShort(PAGE_SIZE);
- fail("IndexOutOfBoundsException expected");
- } catch (Exception e) {
- assertTrue(e instanceof IndexOutOfBoundsException);
- }
- }
-
- // test expected correct behavior
- {
- long seed = random.nextLong();
-
- random.setSeed(seed);
- for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
- segment.putShort(i, (short) random.nextInt());
- }
-
- random.setSeed(seed);
- for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
- assertEquals((short) random.nextInt(), segment.getShort(i));
- }
- }
- }
-
- @Test
- public void testByteBufferWrapping() {
- try {
- MemorySegment seg = new MemorySegment(new byte[1024]);
-
- ByteBuffer buf1 = seg.wrap(13, 47);
- assertEquals(13, buf1.position());
- assertEquals(60, buf1.limit());
- assertEquals(47, buf1.remaining());
-
- ByteBuffer buf2 = seg.wrap(500, 267);
- assertEquals(500, buf2.position());
- assertEquals(767, buf2.limit());
- assertEquals(267, buf2.remaining());
-
- ByteBuffer buf3 = seg.wrap(0, 1024);
- assertEquals(0, buf3.position());
- assertEquals(1024, buf3.limit());
- assertEquals(1024, buf3.remaining());
- }
- catch (Exception e) {
- e.printStackTrace();
- Assert.fail(e.getMessage());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
index 30e417b..0a02f30 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
@@ -16,7 +16,6 @@
* limitations under the License.
*/
-
package org.apache.flink.runtime.operators.drivers;
import org.apache.flink.api.common.ExecutionConfig;
@@ -25,10 +24,10 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.DriverStrategy;
import org.apache.flink.runtime.operators.PactTaskContext;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
@@ -72,7 +71,7 @@ public class TestTaskContext<S, T> implements PactTaskContext<S, T> {
public TestTaskContext() {}
public TestTaskContext(long memoryInBytes) {
- this.memoryManager = new DefaultMemoryManager(memoryInBytes,1 ,32 * 1024, true);
+ this.memoryManager = new MemoryManager(memoryInBytes, 1, 32 * 1024, MemoryType.HEAP, true);
this.taskManageInfo = new TaskManagerRuntimeInfo("localhost", new Configuration());
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java
index e3b697e..abcbf73 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java
@@ -30,6 +30,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.runtime.TupleComparator;
import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.util.MutableObjectIterator;
import org.junit.Test;
@@ -239,7 +240,7 @@ public class CompactingHashTableTest {
private static List<MemorySegment> getMemory(int numSegments, int segmentSize) {
ArrayList<MemorySegment> list = new ArrayList<MemorySegment>(numSegments);
for (int i = 0; i < numSegments; i++) {
- list.add(new MemorySegment(new byte[segmentSize]));
+ list.add(MemorySegmentFactory.allocateUnpooledSegment(segmentSize));
}
return list;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java
index 52f6ffc..d0a6fc6 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java
@@ -36,9 +36,8 @@ import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.hash.MutableHashTable.HashBucketIterator;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.runtime.operators.testutils.UniformIntPairGenerator;
@@ -96,7 +95,7 @@ public class HashTableITCase {
this.pairProbeSideComparator = new IntPairComparator();
this.pairComparator = new IntPairPairComparator();
- this.memManager = new DefaultMemoryManager(32 * 1024 * 1024,1);
+ this.memManager = new MemoryManager(32 * 1024 * 1024,1);
this.ioManager = new IOManagerAsync();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java
index 0c656d6..9cb3606 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypePairComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.operators.hash.MutableHashTable.HashBucketIterator;
@@ -213,7 +214,7 @@ public class HashTablePerformanceComparison {
List<MemorySegment> memory = new ArrayList<MemorySegment>();
for (int i = 0; i < numPages; i++) {
- memory.add(new MemorySegment(new byte[pageSize]));
+ memory.add(MemorySegmentFactory.allocateUnpooledSegment(pageSize));
}
return memory;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableTest.java
index 0d8b81e..0bca22a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableTest.java
@@ -28,6 +28,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.runtime.TupleComparator;
import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.util.MutableObjectIterator;
@@ -153,7 +154,7 @@ public class HashTableTest {
private static List<MemorySegment> getMemory(int numSegments, int segmentSize) {
ArrayList<MemorySegment> list = new ArrayList<MemorySegment>(numSegments);
for (int i = 0; i < numSegments; i++) {
- list.add(new MemorySegment(new byte[segmentSize]));
+ list.add(MemorySegmentFactory.allocateUnpooledSegment(segmentSize));
}
return list;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MemoryHashTableTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MemoryHashTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MemoryHashTableTest.java
index 3dcf688..a71f028 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MemoryHashTableTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MemoryHashTableTest.java
@@ -27,6 +27,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypePairComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.operators.testutils.UniformStringPairGenerator;
import org.apache.flink.runtime.operators.testutils.types.IntList;
import org.apache.flink.runtime.operators.testutils.types.IntListComparator;
@@ -737,7 +738,7 @@ public class MemoryHashTableTest {
List<MemorySegment> memory = new ArrayList<MemorySegment>();
for (int i = 0; i < numPages; i++) {
- memory.add(new MemorySegment(new byte[pageSize]));
+ memory.add(MemorySegmentFactory.allocateUnpooledSegment(pageSize));
}
return memory;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java
index c0f8f59..70c9427 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java
@@ -28,9 +28,8 @@ import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.runtime.operators.testutils.types.StringPair;
import org.apache.flink.runtime.operators.testutils.types.StringPairComparator;
@@ -46,6 +45,7 @@ import org.junit.Test;
import static org.junit.Assert.fail;
public class MutableHashTablePerformanceBenchmark {
+
private static final AbstractInvokable MEM_OWNER = new DummyInvokable();
private MemoryManager memManager;
@@ -68,7 +68,7 @@ public class MutableHashTablePerformanceBenchmark {
this.pairProbeSideComparator = new StringPairComparator();
this.pairComparator = new StringPairPairComparator();
- this.memManager = new DefaultMemoryManager(64 * 1024 * 1024, 1);
+ this.memManager = new MemoryManager(64 * 1024 * 1024, 1);
this.ioManager = new IOManagerAsync();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java
index 0d5a26e..2da97e9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java
@@ -31,8 +31,7 @@ import org.apache.flink.api.java.record.functions.JoinFunction;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.runtime.operators.testutils.TestData;
@@ -104,7 +103,7 @@ public class NonReusingHashMatchIteratorITCase {
this.pairRecordPairComparator = new IntPairRecordPairComparator();
this.recordPairPairComparator = new RecordIntPairPairComparator();
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
this.ioManager = new IOManagerAsync();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java
index 306a370..86f879a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java
@@ -27,12 +27,12 @@ import org.apache.flink.api.common.typeutils.record.RecordPairComparator;
import org.apache.flink.api.common.typeutils.record.RecordSerializer;
import org.apache.flink.api.java.record.functions.JoinFunction;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.hash.HashTableITCase.ConstantsKeyValuePairsIterator;
import org.apache.flink.runtime.operators.hash.MutableHashTable.HashBucketIterator;
import org.apache.flink.runtime.operators.hash.NonReusingHashMatchIteratorITCase.RecordMatch;
@@ -120,7 +120,7 @@ public class NonReusingReOpenableHashTableITCase {
this.recordProbeSideComparator = new RecordComparator(keyPos, keyType);
this.pactRecordComparator = new HashTableITCase.RecordPairComparatorFirstInt();
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, true);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
this.ioManager = new IOManagerAsync();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java
index f770ca4..4fdff76 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java
@@ -38,8 +38,7 @@ import org.apache.flink.api.java.record.functions.JoinFunction;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.runtime.operators.testutils.TestData;
@@ -104,7 +103,7 @@ public class ReusingHashMatchIteratorITCase {
this.pairRecordPairComparator = new IntPairRecordPairComparator();
this.recordPairPairComparator = new RecordIntPairPairComparator();
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
this.ioManager = new IOManagerAsync();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java
index d302487..ba5a325 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java
@@ -37,12 +37,12 @@ import org.apache.flink.api.common.typeutils.record.RecordPairComparator;
import org.apache.flink.api.common.typeutils.record.RecordSerializer;
import org.apache.flink.api.java.record.functions.JoinFunction;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.hash.ReusingHashMatchIteratorITCase.RecordMatch;
import org.apache.flink.runtime.operators.hash.ReusingHashMatchIteratorITCase.RecordMatchRemovingJoin;
import org.apache.flink.runtime.operators.hash.HashTableITCase.ConstantsKeyValuePairsIterator;
@@ -119,7 +119,7 @@ public class ReusingReOpenableHashTableITCase {
this.recordProbeSideComparator = new RecordComparator(keyPos, keyType);
this.pactRecordComparator = new HashTableITCase.RecordPairComparatorFirstInt();
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, true);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
this.ioManager = new IOManagerAsync();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIteratorTest.java
index 41ade9f..bfb7cf2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIteratorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIteratorTest.java
@@ -16,7 +16,6 @@
* limitations under the License.
*/
-
package org.apache.flink.runtime.operators.resettable;
import java.util.ArrayList;
@@ -26,9 +25,7 @@ import org.junit.Assert;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.record.RecordSerializer;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
-import org.apache.flink.runtime.operators.resettable.BlockResettableMutableObjectIterator;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.runtime.operators.testutils.MutableObjectIteratorWrapper;
import org.apache.flink.types.IntValue;
@@ -38,8 +35,8 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
-public class BlockResettableMutableObjectIteratorTest
-{
+public class BlockResettableMutableObjectIteratorTest {
+
private static final int MEMORY_CAPACITY = 3 * 128 * 1024;
private static final int NUM_VALUES = 20000;
@@ -57,7 +54,7 @@ public class BlockResettableMutableObjectIteratorTest
@Before
public void startup() {
// set up IO and memory manager
- this.memman = new DefaultMemoryManager(MEMORY_CAPACITY, 1);
+ this.memman = new MemoryManager(MEMORY_CAPACITY, 1);
// create test objects
this.objects = new ArrayList<Record>(20000);
@@ -83,8 +80,7 @@ public class BlockResettableMutableObjectIteratorTest
}
@Test
- public void testSerialBlockResettableIterator() throws Exception
- {
+ public void testSerialBlockResettableIterator() throws Exception {
try {
// create the resettable Iterator
final BlockResettableMutableObjectIterator<Record> iterator =
@@ -126,8 +122,7 @@ public class BlockResettableMutableObjectIteratorTest
}
@Test
- public void testDoubleBufferedBlockResettableIterator() throws Exception
- {
+ public void testDoubleBufferedBlockResettableIterator() throws Exception {
try {
// create the resettable Iterator
final BlockResettableMutableObjectIterator<Record> iterator =
@@ -170,8 +165,7 @@ public class BlockResettableMutableObjectIteratorTest
}
@Test
- public void testTwelveFoldBufferedBlockResettableIterator() throws Exception
- {
+ public void testTwelveFoldBufferedBlockResettableIterator() throws Exception {
try {
// create the resettable Iterator
final BlockResettableMutableObjectIterator<Record> iterator =
@@ -212,5 +206,4 @@ public class BlockResettableMutableObjectIteratorTest
Assert.fail("Test encountered an exception: " + ex.getMessage());
}
}
-
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java
index 5641f29..d5de75e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java
@@ -16,7 +16,6 @@
* limitations under the License.
*/
-
package org.apache.flink.runtime.operators.resettable;
import java.util.ArrayList;
@@ -27,18 +26,18 @@ import org.junit.Assert;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.record.RecordSerializer;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.types.IntValue;
import org.apache.flink.types.Record;
+
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
-public class NonReusingBlockResettableIteratorTest
-{
+public class NonReusingBlockResettableIteratorTest {
+
private static final int MEMORY_CAPACITY = 3 * 128 * 1024;
private static final int NUM_VALUES = 20000;
@@ -54,7 +53,7 @@ public class NonReusingBlockResettableIteratorTest
@Before
public void startup() {
// set up IO and memory manager
- this.memman = new DefaultMemoryManager(MEMORY_CAPACITY, 1);
+ this.memman = new MemoryManager(MEMORY_CAPACITY, 1);
// create test objects
this.objects = new ArrayList<Record>(20000);
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java
index f0fadfe..7dbd2fb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java
@@ -16,17 +16,16 @@
* limitations under the License.
*/
-
package org.apache.flink.runtime.operators.resettable;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.record.RecordSerializer;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.types.IntValue;
import org.apache.flink.types.Record;
+
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -36,9 +35,8 @@ import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
-
-public class ReusingBlockResettableIteratorTest
-{
+public class ReusingBlockResettableIteratorTest {
+
private static final int MEMORY_CAPACITY = 3 * 128 * 1024;
private static final int NUM_VALUES = 20000;
@@ -54,7 +52,7 @@ public class ReusingBlockResettableIteratorTest
@Before
public void startup() {
// set up IO and memory manager
- this.memman = new DefaultMemoryManager(MEMORY_CAPACITY, 1);
+ this.memman = new MemoryManager(MEMORY_CAPACITY, 1);
// create test objects
this.objects = new ArrayList<Record>(20000);
@@ -197,5 +195,4 @@ public class ReusingBlockResettableIteratorTest
// close the iterator
iterator.close();
}
-
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIteratorTest.java
index 4db520e..0ab9a54 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIteratorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIteratorTest.java
@@ -24,13 +24,14 @@ import java.util.NoSuchElementException;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.base.IntValueSerializer;
+import org.apache.flink.core.memory.MemoryType;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.types.IntValue;
+
import org.junit.Assert;
import org.junit.After;
import org.junit.Before;
@@ -56,7 +57,7 @@ public class SpillingResettableIteratorTest {
@Before
public void startup() {
// set up IO and memory manager
- this.memman = new DefaultMemoryManager(MEMORY_CAPACITY, 32 * 1024);
+ this.memman = new MemoryManager(MEMORY_CAPACITY, 1, 32 * 1024, MemoryType.HEAP, true);
this.ioman = new IOManagerAsync();
// create test objects
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIteratorTest.java
index fa72bbf..c64db54 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIteratorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIteratorTest.java
@@ -16,7 +16,6 @@
* limitations under the License.
*/
-
package org.apache.flink.runtime.operators.resettable;
import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -24,8 +23,7 @@ import org.apache.flink.api.common.typeutils.record.RecordSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.runtime.operators.testutils.MutableObjectIteratorWrapper;
import org.apache.flink.types.IntValue;
@@ -56,7 +54,7 @@ public class SpillingResettableMutableObjectIteratorTest {
@Before
public void startup() {
// set up IO and memory manager
- this.memman = new DefaultMemoryManager(MEMORY_CAPACITY, 32 * 1024);
+ this.memman = new MemoryManager(MEMORY_CAPACITY, 1);
this.ioman = new IOManagerAsync();
// create test objects
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java
index d4da9d3..0c0e836 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java
@@ -36,8 +36,7 @@ import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
import org.apache.flink.runtime.operators.testutils.CollectionIterator;
import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
@@ -53,6 +52,7 @@ import org.apache.flink.runtime.operators.testutils.TestData.TupleGeneratorItera
import org.apache.flink.runtime.util.ResettableMutableObjectIterator;
import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
+
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -103,7 +103,7 @@ public abstract class AbstractSortMergeOuterJoinIteratorITCase {
comparator2 = typeInfo2.createComparator(new int[]{0}, new boolean[]{true}, 0, config);
pairComp = new GenericPairComparator<>(comparator1, comparator2);
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
this.ioManager = new IOManagerAsync();
}
@@ -299,7 +299,7 @@ public abstract class AbstractSortMergeOuterJoinIteratorITCase {
TypePairComparator<Tuple2<Integer, String>, Tuple2<Integer, String>> pairComparator = new GenericPairComparator<>(comparator1, comparator2);
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
this.ioManager = new IOManagerAsync();
final int DUPLICATE_KEY = 13;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
index 0ba9823..75593b8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
@@ -37,8 +37,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.runtime.operators.testutils.TestData;
import org.apache.flink.runtime.operators.testutils.TestData.Key;
@@ -80,7 +79,7 @@ public class CombiningUnilateralSortMergerITCase {
@SuppressWarnings("unchecked")
@Before
public void beforeTest() {
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
this.ioManager = new IOManagerAsync();
this.serializerFactory = RecordSerializerFactory.get();
@@ -190,7 +189,7 @@ public class CombiningUnilateralSortMergerITCase {
{
final Hashtable<TestData.Key, Integer> countTable = new Hashtable<TestData.Key, Integer>(KEY_MAX);
for (int i = 1; i <= KEY_MAX; i++) {
- countTable.put(new TestData.Key(i), Integer.valueOf(0));
+ countTable.put(new TestData.Key(i), 0);
}
// comparator
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortITCase.java
index 46a1632..5aa9efb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortITCase.java
@@ -27,8 +27,7 @@ import org.apache.flink.api.common.typeutils.record.RecordSerializerFactory;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.runtime.operators.testutils.RandomIntPairGenerator;
import org.apache.flink.runtime.operators.testutils.TestData;
@@ -82,7 +81,7 @@ public class ExternalSortITCase {
@SuppressWarnings("unchecked")
@Before
public void beforeTest() {
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
this.ioManager = new IOManagerAsync();
this.pactRecordSerializer = RecordSerializerFactory.get();
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java
index 86be614..951ce30 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java
@@ -32,8 +32,7 @@ import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.testutils.DummyInvokable;
import org.apache.flink.util.MutableObjectIterator;
import org.junit.After;
@@ -66,7 +65,7 @@ public class ExternalSortLargeRecordsITCase {
@Before
public void beforeTest() {
- this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+ this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
this.ioManager = new IOManagerAsync();
}
[15/15] flink git commit: [FLINK-1320] [core] Add an off-heap variant
of the managed memory
Posted by se...@apache.org.
[FLINK-1320] [core] Add an off-heap variant of the managed memory
This closes #1093
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/655a891d
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/655a891d
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/655a891d
Branch: refs/heads/master
Commit: 655a891d929db9d858bb5c2edf54419f2b0d3ace
Parents: 1800434
Author: Stephan Ewen <se...@apache.org>
Authored: Sun Aug 30 22:36:46 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Sep 8 20:58:05 2015 +0200
----------------------------------------------------------------------
.../flink/configuration/ConfigConstants.java | 14 +-
.../flink/core/memory/HeapMemorySegment.java | 203 ++
.../flink/core/memory/HybridMemorySegment.java | 466 ++++
.../apache/flink/core/memory/MemorySegment.java | 938 ++++---
.../flink/core/memory/MemorySegmentFactory.java | 211 ++
.../apache/flink/core/memory/MemoryType.java | 35 +
.../apache/flink/core/memory/MemoryUtils.java | 34 +-
.../common/typeutils/ComparatorTestBase.java | 10 +-
.../flink/core/memory/CrossSegmentTypeTest.java | 356 +++
.../core/memory/EndiannessAccessChecks.java | 183 ++
.../core/memory/HeapMemorySegmentTest.java | 71 +
.../memory/HybridOffHeapMemorySegmentTest.java | 84 +
.../memory/HybridOnHeapMemorySegmentTest.java | 82 +
.../core/memory/MemorySegmentChecksTest.java | 135 +
.../core/memory/MemorySegmentTestBase.java | 2571 ++++++++++++++++++
.../memory/MemorySegmentUndersizedTest.java | 1367 ++++++++++
.../memory/OperationsOnFreedSegmentTest.java | 1195 ++++++++
.../benchmarks/CoreMemorySegmentOutView.java | 360 +++
.../LongSerializationSpeedBenchmark.java | 232 ++
.../benchmarks/MemorySegmentSpeedBenchmark.java | 1633 +++++++++++
.../benchmarks/PureHeapMemorySegment.java | 466 ++++
.../PureHeapMemorySegmentOutView.java | 359 +++
.../benchmarks/PureHybridMemorySegment.java | 887 ++++++
.../PureHybridMemorySegmentOutView.java | 359 +++
.../benchmarks/PureOffHeapMemorySegment.java | 790 ++++++
.../PureOffHeapMemorySegmentOutView.java | 359 +++
.../StringSerializationSpeedBenchmark.java | 207 ++
.../apache/flink/types/NormalizableKeyTest.java | 26 +-
.../flink/runtime/execution/Environment.java | 2 +-
.../runtime/io/disk/FileChannelInputView.java | 4 +-
.../runtime/io/disk/FileChannelOutputView.java | 4 +-
.../runtime/io/disk/RandomAccessInputView.java | 2 +-
.../runtime/io/disk/RandomAccessOutputView.java | 2 +-
.../io/disk/SeekableFileChannelInputView.java | 4 +-
.../io/disk/SimpleCollectingOutputView.java | 2 +-
.../flink/runtime/io/disk/SpillingBuffer.java | 2 +-
.../disk/iomanager/ChannelReaderInputView.java | 2 +-
.../disk/iomanager/ChannelWriterOutputView.java | 2 +-
.../runtime/io/network/NetworkEnvironment.java | 3 +-
.../AdaptiveSpanningRecordDeserializer.java | 6 +-
.../api/serialization/EventSerializer.java | 6 +-
...llingAdaptiveSpanningRecordDeserializer.java | 6 +-
.../flink/runtime/io/network/buffer/Buffer.java | 5 +-
.../io/network/buffer/NetworkBufferPool.java | 30 +-
.../netty/PartitionRequestClientHandler.java | 4 +-
.../SpilledSubpartitionViewSyncIO.java | 3 +-
.../iterative/io/SerializedUpdateBuffer.java | 4 +-
.../runtime/memory/AbstractPagedInputView.java | 568 ++++
.../runtime/memory/AbstractPagedOutputView.java | 414 +++
.../runtime/memory/ListMemorySegmentSource.java | 48 +
.../memory/MemoryAllocationException.java | 44 +
.../flink/runtime/memory/MemoryManager.java | 700 +++++
.../memorymanager/AbstractPagedInputView.java | 566 ----
.../memorymanager/AbstractPagedOutputView.java | 416 ---
.../memorymanager/CheckedMemorySegment.java | 407 ---
.../memorymanager/DefaultMemoryManager.java | 490 ----
.../memorymanager/ListMemorySegmentSource.java | 48 -
.../MemoryAllocationException.java | 44 -
.../runtime/memorymanager/MemoryManager.java | 134 -
.../memorymanager/SimpleMemorySegment.java | 329 ---
.../memorymanager/UnsafeMemorySegment.java | 391 ---
.../operators/AbstractOuterJoinDriver.java | 2 +-
.../flink/runtime/operators/CrossDriver.java | 2 +-
.../runtime/operators/FullOuterJoinDriver.java | 2 +-
.../operators/GroupReduceCombineDriver.java | 2 +-
.../flink/runtime/operators/JoinDriver.java | 2 +-
.../runtime/operators/LeftOuterJoinDriver.java | 2 +-
.../runtime/operators/PactTaskContext.java | 2 +-
.../runtime/operators/ReduceCombineDriver.java | 2 +-
.../runtime/operators/RegularPactTask.java | 2 +-
.../runtime/operators/RightOuterJoinDriver.java | 2 +-
.../flink/runtime/operators/TempBarrier.java | 6 +-
.../chaining/GroupCombineChainedDriver.java | 2 +-
.../SynchronousChainedCombineDriver.java | 2 +-
.../operators/hash/CompactingHashTable.java | 2 +-
.../operators/hash/HashMatchIteratorBase.java | 4 +-
.../runtime/operators/hash/HashPartition.java | 4 +-
.../operators/hash/InMemoryPartition.java | 6 +-
.../NonReusingBuildFirstHashMatchIterator.java | 4 +-
...ngBuildFirstReOpenableHashMatchIterator.java | 4 +-
.../NonReusingBuildSecondHashMatchIterator.java | 4 +-
...gBuildSecondReOpenableHashMatchIterator.java | 4 +-
.../ReusingBuildFirstHashMatchIterator.java | 4 +-
...ngBuildFirstReOpenableHashMatchIterator.java | 4 +-
.../ReusingBuildSecondHashMatchIterator.java | 4 +-
...gBuildSecondReOpenableHashMatchIterator.java | 4 +-
.../AbstractBlockResettableIterator.java | 6 +-
.../BlockResettableMutableObjectIterator.java | 4 +-
.../NonReusingBlockResettableIterator.java | 4 +-
.../ReusingBlockResettableIterator.java | 4 +-
.../resettable/SpillingResettableIterator.java | 6 +-
...SpillingResettableMutableObjectIterator.java | 6 +-
.../sort/AbstractMergeInnerJoinIterator.java | 4 +-
.../operators/sort/AbstractMergeIterator.java | 4 +-
.../sort/AbstractMergeOuterJoinIterator.java | 4 +-
.../sort/CombiningUnilateralSortMerger.java | 4 +-
.../operators/sort/FixedLengthRecordSorter.java | 12 +-
.../operators/sort/LargeRecordHandler.java | 2 +-
.../sort/NonReusingMergeInnerJoinIterator.java | 4 +-
.../sort/NonReusingMergeOuterJoinIterator.java | 4 +-
.../operators/sort/NormalizedKeySorter.java | 18 +-
.../sort/ReusingMergeInnerJoinIterator.java | 4 +-
.../sort/ReusingMergeOuterJoinIterator.java | 4 +-
.../operators/sort/UnilateralSortMerger.java | 4 +-
.../operators/util/CoGroupTaskIterator.java | 2 +-
.../operators/util/JoinTaskIterator.java | 2 +-
.../runtime/taskmanager/RuntimeEnvironment.java | 2 +-
.../apache/flink/runtime/taskmanager/Task.java | 2 +-
.../runtime/util/EnvironmentInformation.java | 19 +-
.../NetworkEnvironmentConfiguration.scala | 4 +-
.../flink/runtime/taskmanager/TaskManager.scala | 99 +-
.../flink/runtime/io/disk/ChannelViewsTest.java | 6 +-
.../io/disk/FileChannelStreamsITCase.java | 7 +-
.../runtime/io/disk/FileChannelStreamsTest.java | 8 +-
.../disk/SeekableFileChannelInputViewTest.java | 24 +-
.../runtime/io/disk/SpillingBufferTest.java | 31 +-
.../AsynchronousFileIOChannelTest.java | 8 +-
.../BufferFileWriterFileSegmentReaderTest.java | 7 +-
.../iomanager/BufferFileWriterReaderTest.java | 4 +-
.../io/disk/iomanager/IOManagerAsyncTest.java | 5 +-
.../io/disk/iomanager/IOManagerITCase.java | 8 +-
.../IOManagerPerformanceBenchmark.java | 6 +-
.../io/network/NetworkEnvironmentTest.java | 4 +-
.../api/serialization/PagedViewsTest.java | 15 +-
.../SpanningRecordSerializationTest.java | 4 +-
.../SpanningRecordSerializerTest.java | 7 +-
.../io/network/api/writer/RecordWriterTest.java | 9 +-
.../network/buffer/BufferPoolFactoryTest.java | 3 +-
.../runtime/io/network/buffer/BufferTest.java | 5 +-
.../io/network/buffer/LocalBufferPoolTest.java | 3 +-
.../network/buffer/NetworkBufferPoolTest.java | 5 +-
.../netty/NettyMessageSerializationTest.java | 4 +-
.../IteratorWrappingTestSingleInputGate.java | 4 +-
.../consumer/LocalInputChannelTest.java | 3 +-
.../partition/consumer/SingleInputGateTest.java | 5 +-
.../network/serialization/LargeRecordsTest.java | 6 +-
.../io/network/util/TestBufferFactory.java | 6 +-
.../memory/MemoryManagerLazyAllocationTest.java | 7 +-
.../flink/runtime/memory/MemoryManagerTest.java | 7 +-
.../runtime/memory/MemorySegmentSimpleTest.java | 576 ++++
.../memory/MemorySegmentSpeedBenchmark.java | 867 ------
.../flink/runtime/memory/MemorySegmentTest.java | 575 ----
.../operators/drivers/TestTaskContext.java | 7 +-
.../operators/hash/CompactingHashTableTest.java | 3 +-
.../runtime/operators/hash/HashTableITCase.java | 7 +-
.../hash/HashTablePerformanceComparison.java | 3 +-
.../runtime/operators/hash/HashTableTest.java | 3 +-
.../operators/hash/MemoryHashTableTest.java | 3 +-
.../MutableHashTablePerformanceBenchmark.java | 8 +-
.../hash/NonReusingHashMatchIteratorITCase.java | 5 +-
.../NonReusingReOpenableHashTableITCase.java | 8 +-
.../hash/ReusingHashMatchIteratorITCase.java | 5 +-
.../hash/ReusingReOpenableHashTableITCase.java | 8 +-
...lockResettableMutableObjectIteratorTest.java | 21 +-
.../NonReusingBlockResettableIteratorTest.java | 11 +-
.../ReusingBlockResettableIteratorTest.java | 13 +-
.../SpillingResettableIteratorTest.java | 7 +-
...lingResettableMutableObjectIteratorTest.java | 6 +-
...bstractSortMergeOuterJoinIteratorITCase.java | 8 +-
.../CombiningUnilateralSortMergerITCase.java | 7 +-
.../operators/sort/ExternalSortITCase.java | 5 +-
.../sort/ExternalSortLargeRecordsITCase.java | 5 +-
.../sort/FixedLengthRecordSorterTest.java | 14 +-
.../sort/LargeRecordHandlerITCase.java | 28 +-
.../operators/sort/LargeRecordHandlerTest.java | 21 +-
...ReusingSortMergeInnerJoinIteratorITCase.java | 17 +-
...ReusingSortMergeOuterJoinIteratorITCase.java | 2 +-
.../operators/sort/NormalizedKeySorterTest.java | 15 +-
...ReusingSortMergeInnerJoinIteratorITCase.java | 17 +-
...ReusingSortMergeOuterJoinIteratorITCase.java | 2 +-
.../testutils/BinaryOperatorTestBase.java | 5 +-
.../operators/testutils/DriverTestBase.java | 5 +-
.../operators/testutils/MockEnvironment.java | 9 +-
.../operators/testutils/TaskTestBase.java | 2 +-
.../testutils/UnaryOperatorTestBase.java | 5 +-
.../runtime/operators/util/BloomFilterTest.java | 5 +-
.../operators/util/HashVsSortMiniBenchmark.java | 6 +-
.../runtime/taskmanager/TaskAsyncCallTest.java | 2 +-
...askManagerComponentsStartupShutdownTest.java | 8 +-
.../flink/runtime/taskmanager/TaskTest.java | 2 +-
.../util/DataInputOutputSerializerTest.java | 4 +-
.../testingUtils/TestingTaskManager.scala | 4 +-
.../streaming/runtime/io/BufferSpiller.java | 3 +-
.../consumer/StreamTestSingleInputGate.java | 18 +-
.../io/BarrierBufferMassiveRandomTest.java | 5 +-
.../streaming/runtime/io/BarrierBufferTest.java | 8 +-
.../runtime/io/BarrierTrackerTest.java | 4 +-
.../streaming/runtime/io/BufferSpillerTest.java | 3 +-
.../runtime/io/StreamRecordWriterTest.java | 7 +-
.../runtime/tasks/StreamMockEnvironment.java | 11 +-
.../runtime/tasks/StreamTaskTestHarness.java | 2 +-
.../tez/runtime/TezRuntimeEnvironment.java | 11 +-
.../org/apache/flink/tez/runtime/TezTask.java | 2 +-
.../HashTableRecordWidthCombinations.java | 3 +-
.../flink/test/manual/MassiveStringSorting.java | 8 +-
.../test/manual/MassiveStringValueSorting.java | 19 +-
.../misc/MassiveCaseClassSortingITCase.scala | 4 +-
.../scala/runtime/CaseClassComparatorTest.scala | 30 +-
.../org/apache/flink/yarn/YarnTaskManager.scala | 4 +-
pom.xml | 2 +-
tools/maven/checkstyle.xml | 2 +-
201 files changed, 16226 insertions(+), 5096 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index 9ec71d2..bbaf71a 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -125,9 +125,14 @@ public final class ConfigConstants {
public static final String TASK_MANAGER_MEMORY_FRACTION_KEY = "taskmanager.memory.fraction";
/**
- * The key for the config parameter defining whether the memory manager allocates memory lazy.
+ * The fraction of off-heap memory relative to the heap size.
*/
- public static final String TASK_MANAGER_MEMORY_LAZY_ALLOCATION_KEY = "taskmanager.memory.lazyalloc";
+ public static final String TASK_MANAGER_MEMORY_OFF_HEAP_RATIO_KEY = "taskmanager.memory.off-heap-ratio";
+
+ /**
+ * The config parameter defining the memory allocation method (JVM heap or off-heap).
+ */
+ public static final String TASK_MANAGER_MEMORY_OFF_HEAP_KEY = "taskmanager.memory.off-heap";
/**
* The config parameter defining the number of buffers used in the network stack. This defines the
@@ -543,6 +548,11 @@ public final class ConfigConstants {
public static final float DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION = 0.7f;
/**
+ * The default ratio of heap to off-heap memory, when the TaskManager is started with off-heap memory.
+ */
+ public static final float DEFAULT_MEMORY_MANAGER_MEMORY_OFF_HEAP_RATIO = 3.0f;
+
+ /**
* Default number of buffers used in the network stack.
*/
public static final int DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS = 2048;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/main/java/org/apache/flink/core/memory/HeapMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/HeapMemorySegment.java b/flink-core/src/main/java/org/apache/flink/core/memory/HeapMemorySegment.java
new file mode 100644
index 0000000..0685d59
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/HeapMemorySegment.java
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+/**
+ * This class represents a piece of heap memory managed by Flink.
+ * The segment is backed by a byte array and features random put and get methods for the basic types,
+ * as well as compare and swap methods.
+ * <p>
+ * This class specialized byte access and byte copy calls for heap memory, while reusing the
+ * multi-byte type accesses and cross-segment operations from the MemorySegment.
+ * <p>
+ * Note that memory segments should usually not be allocated manually, but rather through the
+ * {@link MemorySegmentFactory}.
+ */
+public final class HeapMemorySegment extends MemorySegment {
+
+ /** An extra reference to the heap memory, so we can let byte array checks fail
+ * by the built-in checks automatically without extra checks */
+ private byte[] memory;
+
+ /**
+ * Creates a new memory segment that represents the data in the given byte array.
+ * The owner of this memory segment is null.
+ *
+ * @param memory The byte array that holds the data.
+ */
+ HeapMemorySegment(byte[] memory) {
+ this(memory, null);
+ }
+
+ /**
+ * Creates a new memory segment that represents the data in the given byte array.
+ * The memory segment references the given owner.
+ *
+ * @param memory The byte array that holds the data.
+ * @param owner The owner referenced by the memory segment.
+ */
+ HeapMemorySegment(byte[] memory, Object owner) {
+ super(Objects.requireNonNull(memory), owner);
+ this.memory = memory;
+ }
+
+ // -------------------------------------------------------------------------
+ // MemorySegment operations
+ // -------------------------------------------------------------------------
+
+ @Override
+ public void free() {
+ super.free();
+ this.memory = null;
+ }
+
+ @Override
+ public ByteBuffer wrap(int offset, int length) {
+ try {
+ return ByteBuffer.wrap(this.memory, offset, length);
+ }
+ catch (NullPointerException e) {
+ throw new IllegalStateException("segment has been freed");
+ }
+ }
+
+ /**
+ * Gets the byte array that backs this memory segment.
+ *
+ * @return The byte array that backs this memory segment, or null, if the segment has been freed.
+ */
+ public byte[] getArray() {
+ return this.heapMemory;
+ }
+
+ // ------------------------------------------------------------------------
+ // Random Access get() and put() methods
+ // ------------------------------------------------------------------------
+
+ @Override
+ public final byte get(int index) {
+ return this.memory[index];
+ }
+
+ @Override
+ public final void put(int index, byte b) {
+ this.memory[index] = b;
+ }
+
+ @Override
+ public final void get(int index, byte[] dst) {
+ get(index, dst, 0, dst.length);
+ }
+
+ @Override
+ public final void put(int index, byte[] src) {
+ put(index, src, 0, src.length);
+ }
+
+ @Override
+ public final void get(int index, byte[] dst, int offset, int length) {
+ // system arraycopy does the boundary checks anyways, no need to check extra
+ System.arraycopy(this.memory, index, dst, offset, length);
+ }
+
+ @Override
+ public final void put(int index, byte[] src, int offset, int length) {
+ // system arraycopy does the boundary checks anyways, no need to check extra
+ System.arraycopy(src, offset, this.memory, index, length);
+ }
+
+ @Override
+ public final boolean getBoolean(int index) {
+ return this.memory[index] != 0;
+ }
+
+ @Override
+ public final void putBoolean(int index, boolean value) {
+ this.memory[index] = (byte) (value ? 1 : 0);
+ }
+
+ // -------------------------------------------------------------------------
+ // Bulk Read and Write Methods
+ // -------------------------------------------------------------------------
+
+ @Override
+ public final void get(DataOutput out, int offset, int length) throws IOException {
+ out.write(this.memory, offset, length);
+ }
+
+ @Override
+ public final void put(DataInput in, int offset, int length) throws IOException {
+ in.readFully(this.memory, offset, length);
+ }
+
+ @Override
+ public final void get(int offset, ByteBuffer target, int numBytes) {
+ // ByteBuffer performs the boundary checks
+ target.put(this.memory, offset, numBytes);
+ }
+
+ @Override
+ public final void put(int offset, ByteBuffer source, int numBytes) {
+ // ByteBuffer performs the boundary checks
+ source.get(this.memory, offset, numBytes);
+ }
+
+ // -------------------------------------------------------------------------
+ // Factoring
+ // -------------------------------------------------------------------------
+
+ /**
+ * A memory segment factory that produces heap memory segments. Note that this factory does not
+ * support to allocate off-heap memory.
+ */
+ public static final class HeapMemorySegmentFactory implements MemorySegmentFactory.Factory {
+
+ @Override
+ public HeapMemorySegment wrap(byte[] memory) {
+ return new HeapMemorySegment(memory);
+ }
+
+ @Override
+ public HeapMemorySegment allocateUnpooledSegment(int size, Object owner) {
+ return new HeapMemorySegment(new byte[size], owner);
+ }
+
+ @Override
+ public HeapMemorySegment wrapPooledHeapMemory(byte[] memory, Object owner) {
+ return new HeapMemorySegment(memory, owner);
+ }
+
+ @Override
+ public HeapMemorySegment wrapPooledOffHeapMemory(ByteBuffer memory, Object owner) {
+ throw new UnsupportedOperationException(
+ "The MemorySegment factory was not initialized for off-heap memory.");
+ }
+
+ /** prevent external instantiation */
+ HeapMemorySegmentFactory() {}
+ };
+
+ public static final HeapMemorySegmentFactory FACTORY = new HeapMemorySegmentFactory();
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java b/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java
new file mode 100644
index 0000000..f68723b
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java
@@ -0,0 +1,466 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.nio.BufferOverflowException;
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+
+/**
+ * This class represents a piece of memory managed by Flink. The memory can be on-heap or off-heap,
+ * this is transparently handled by this class.
+ * <p>
+ * This class specialized byte access and byte copy calls for heap memory, while reusing the
+ * multi-byte type accesses and cross-segment operations from the MemorySegment.
+ * <p>
+ * This class subsumes the functionality of the {@link org.apache.flink.core.memory.HeapMemorySegment},
+ * but is a bit less efficient for operations on individual bytes.
+ * <p>
+ * Note that memory segments should usually not be allocated manually, but rather through the
+ * {@link MemorySegmentFactory}.
+ */
+public final class HybridMemorySegment extends MemorySegment {
+
+ /** The direct byte buffer that allocated the off-heap memory. This memory segment holds a reference
+ * to that buffer, so as long as this memory segment lives, the memory will not be released. */
+ private final ByteBuffer offHeapBuffer;
+
+ /**
+ * Creates a new memory segment that represents the memory backing the given direct byte buffer.
+ * Note that the given ByteBuffer must be direct {@link java.nio.ByteBuffer#allocateDirect(int)},
+ * otherwise this method with throw an IllegalArgumentException.
+ * <p>
+ * The owner referenced by this memory segment is null.
+ *
+ * @param buffer The byte buffer whose memory is represented by this memory segment.
+ * @throws IllegalArgumentException Thrown, if the given ByteBuffer is not direct.
+ */
+ HybridMemorySegment(ByteBuffer buffer) {
+ this(buffer, null);
+ }
+
+ /**
+ * Creates a new memory segment that represents the memory backing the given direct byte buffer.
+ * Note that the given ByteBuffer must be direct {@link java.nio.ByteBuffer#allocateDirect(int)},
+ * otherwise this method with throw an IllegalArgumentException.
+ * <p>
+ * The memory segment references the given owner.
+ *
+ * @param buffer The byte buffer whose memory is represented by this memory segment.
+ * @param owner The owner references by this memory segment.
+ * @throws IllegalArgumentException Thrown, if the given ByteBuffer is not direct.
+ */
+ HybridMemorySegment(ByteBuffer buffer, Object owner) {
+ super(checkBufferAndGetAddress(buffer), buffer.capacity(), owner);
+ this.offHeapBuffer = buffer;
+ }
+
+ /**
+ * Creates a new memory segment that represents the memory of the byte array.
+ * <p>
+ * The owner referenced by this memory segment is null.
+ *
+ * @param buffer The byte array whose memory is represented by this memory segment.
+ */
+ HybridMemorySegment(byte[] buffer) {
+ this(buffer, null);
+ }
+
+ /**
+ * Creates a new memory segment that represents the memory of the byte array.
+ * <p>
+ * The memory segment references the given owner.
+ *
+ * @param buffer The byte array whose memory is represented by this memory segment.
+ * @param owner The owner references by this memory segment.
+ */
+ HybridMemorySegment(byte[] buffer, Object owner) {
+ super(buffer, owner);
+ this.offHeapBuffer = null;
+ }
+
+ // -------------------------------------------------------------------------
+ // MemorySegment operations
+ // -------------------------------------------------------------------------
+
+ public byte[] getArray() {
+ if (heapMemory != null) {
+ return heapMemory;
+ } else {
+ throw new IllegalStateException("Memory segment does not represent heap memory");
+ }
+ }
+
+ /**
+ * Gets the buffer that owns the memory of this memory segment.
+ *
+ * @return The byte buffer that owns the memory of this memory segment.
+ */
+ public ByteBuffer getOffHeapBuffer() {
+ if (offHeapBuffer != null) {
+ return offHeapBuffer;
+ } else {
+ throw new IllegalStateException("Memory segment does not represent off heap memory");
+ }
+ }
+
+ @Override
+ public ByteBuffer wrap(int offset, int length) {
+ if (address <= addressLimit) {
+ if (heapMemory != null) {
+ return ByteBuffer.wrap(heapMemory, offset, length);
+ }
+ else {
+ try {
+ ByteBuffer wrapper = offHeapBuffer.duplicate();
+ wrapper.limit(offset + length);
+ wrapper.position(offset);
+ return wrapper;
+ }
+ catch (IllegalArgumentException e) {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+ }
+ else {
+ throw new IllegalStateException("segment has been freed");
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Random Access get() and put() methods
+ // ------------------------------------------------------------------------
+
+ @Override
+ public byte get(int index) {
+ final long pos = address + index;
+ if (index >= 0 && pos < addressLimit) {
+ return UNSAFE.getByte(heapMemory, pos);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("segment has been freed");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ @Override
+ public void put(int index, byte b) {
+ final long pos = address + index;
+ if (index >= 0 && pos < addressLimit) {
+ UNSAFE.putByte(heapMemory, pos, b);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("segment has been freed");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ @Override
+ public void get(int index, byte[] dst) {
+ get(index, dst, 0, dst.length);
+ }
+
+ @Override
+ public void put(int index, byte[] src) {
+ put(index, src, 0, src.length);
+ }
+
+ @Override
+ public void get(int index, byte[] dst, int offset, int length) {
+ // check the byte array offset and length and the status
+ if ( (offset | length | (offset + length) | (dst.length - (offset + length))) < 0) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - length) {
+ final long arrayAddress = BYTE_ARRAY_BASE_OFFSET + offset;
+ UNSAFE.copyMemory(heapMemory, pos, dst, arrayAddress, length);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("segment has been freed");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ @Override
+ public void put(int index, byte[] src, int offset, int length) {
+ // check the byte array offset and length
+ if ((offset | length | (offset + length) | (src.length - (offset + length))) < 0) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ final long pos = address + index;
+
+ if (index >= 0 && pos <= addressLimit - length) {
+ final long arrayAddress = BYTE_ARRAY_BASE_OFFSET + offset;
+ UNSAFE.copyMemory(src, arrayAddress, heapMemory, pos, length);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("segment has been freed");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ @Override
+ public boolean getBoolean(int index) {
+ return get(index) != 0;
+ }
+
+ @Override
+ public void putBoolean(int index, boolean value) {
+ put(index, (byte) (value ? 1 : 0));
+ }
+
+ // -------------------------------------------------------------------------
+ // Bulk Read and Write Methods
+ // -------------------------------------------------------------------------
+
+ @Override
+ public final void get(DataOutput out, int offset, int length) throws IOException {
+ if (address <= addressLimit) {
+ if (heapMemory != null) {
+ out.write(heapMemory, offset, length);
+ }
+ else {
+ while (length >= 8) {
+ out.writeLong(getLongBigEndian(offset));
+ offset += 8;
+ length -= 8;
+ }
+
+ while (length > 0) {
+ out.writeByte(get(offset));
+ offset++;
+ length--;
+ }
+ }
+ }
+ else {
+ throw new IllegalStateException("segment has been freed");
+ }
+ }
+
+ @Override
+ public final void put(DataInput in, int offset, int length) throws IOException {
+ if (address <= addressLimit) {
+ if (heapMemory != null) {
+ in.readFully(heapMemory, offset, length);
+ }
+ else {
+ while (length >= 8) {
+ putLongBigEndian(offset, in.readLong());
+ offset += 8;
+ length -= 8;
+ }
+ while (length > 0) {
+ put(offset, in.readByte());
+ offset++;
+ length--;
+ }
+ }
+ }
+ else {
+ throw new IllegalStateException("segment has been freed");
+ }
+ }
+
+ @Override
+ public final void get(int offset, ByteBuffer target, int numBytes) {
+ // check the byte array offset and length
+ if ((offset | numBytes | (offset + numBytes)) < 0) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ final int targetOffset = target.position();
+ final int remaining = target.remaining();
+
+ if (remaining < numBytes) {
+ throw new BufferOverflowException();
+ }
+
+ if (target.isDirect()) {
+ // copy to the target memory directly
+ final long targetPointer = getAddress(target) + targetOffset;
+ final long sourcePointer = address + offset;
+
+ if (sourcePointer <= addressLimit - numBytes) {
+ UNSAFE.copyMemory(heapMemory, sourcePointer, null, targetPointer, numBytes);
+ target.position(targetOffset + numBytes);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("segment has been freed");
+ }
+ else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+ else if (target.hasArray()) {
+ // move directly into the byte array
+ get(offset, target.array(), targetOffset + target.arrayOffset(), numBytes);
+
+ // this must be after the get() call to ensue that the byte buffer is not
+ // modified in case the call fails
+ target.position(targetOffset + numBytes);
+ }
+ else {
+ // neither heap buffer nor direct buffer
+ while (target.hasRemaining()) {
+ target.put(get(offset++));
+ }
+ }
+ }
+
+ @Override
+ public final void put(int offset, ByteBuffer source, int numBytes) {
+ // check the byte array offset and length
+ if ((offset | numBytes | (offset + numBytes)) < 0) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ final int sourceOffset = source.position();
+ final int remaining = source.remaining();
+
+ if (remaining < numBytes) {
+ throw new BufferUnderflowException();
+ }
+
+ if (source.isDirect()) {
+ // copy to the target memory directly
+ final long sourcePointer = getAddress(source) + sourceOffset;
+ final long targetPointer = address + offset;
+
+ if (targetPointer <= addressLimit - numBytes) {
+ UNSAFE.copyMemory(null, sourcePointer, heapMemory, targetPointer, numBytes);
+ source.position(sourceOffset + numBytes);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("segment has been freed");
+ }
+ else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+ else if (source.hasArray()) {
+ // move directly into the byte array
+ put(offset, source.array(), sourceOffset + source.arrayOffset(), numBytes);
+
+ // this must be after the get() call to ensue that the byte buffer is not
+ // modified in case the call fails
+ source.position(sourceOffset + numBytes);
+ }
+ else {
+ // neither heap buffer nor direct buffer
+ while (source.hasRemaining()) {
+ put(offset++, source.get());
+ }
+ }
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Utilities for native memory accesses and checks
+ // --------------------------------------------------------------------------------------------
+
+ /** The reflection fields with which we access the off-heap pointer from direct ByteBuffers */
+ private static final Field ADDRESS_FIELD;
+
+ static {
+ try {
+ ADDRESS_FIELD = java.nio.Buffer.class.getDeclaredField("address");
+ ADDRESS_FIELD.setAccessible(true);
+ }
+ catch (Throwable t) {
+ throw new RuntimeException(
+ "Cannot initialize HybridMemorySegment: off-heap memory is incompatible with this JVM.", t);
+ }
+ }
+
+ private static long getAddress(ByteBuffer buffer) {
+ if (buffer == null) {
+ throw new NullPointerException("buffer is null");
+ }
+ try {
+ return (Long) ADDRESS_FIELD.get(buffer);
+ }
+ catch (Throwable t) {
+ throw new RuntimeException("Could not access direct byte buffer address.", t);
+ }
+ }
+
+ private static long checkBufferAndGetAddress(ByteBuffer buffer) {
+ if (buffer == null) {
+ throw new NullPointerException("buffer is null");
+ }
+ if (!buffer.isDirect()) {
+ throw new IllegalArgumentException("Can't initialize from non-direct ByteBuffer.");
+ }
+ return getAddress(buffer);
+ }
+
+ // -------------------------------------------------------------------------
+ // Factoring
+ // -------------------------------------------------------------------------
+
+ /**
+ * Base factory for hybrid memory segments.
+ */
+ public static final class HybridMemorySegmentFactory implements MemorySegmentFactory.Factory {
+
+ @Override
+ public HybridMemorySegment wrap(byte[] memory) {
+ return new HybridMemorySegment(memory);
+ }
+
+ @Override
+ public HybridMemorySegment allocateUnpooledSegment(int size, Object owner) {
+ return new HybridMemorySegment(new byte[size], owner);
+ }
+
+ @Override
+ public HybridMemorySegment wrapPooledHeapMemory(byte[] memory, Object owner) {
+ return new HybridMemorySegment(memory, owner);
+ }
+
+ @Override
+ public HybridMemorySegment wrapPooledOffHeapMemory(ByteBuffer memory, Object owner) {
+ return new HybridMemorySegment(memory, owner);
+ }
+
+ /** prevent external instantiation */
+ HybridMemorySegmentFactory() {}
+ };
+
+ public static final HybridMemorySegmentFactory FACTORY = new HybridMemorySegmentFactory();
+}
[11/15] flink git commit: [FLINK-1320] [core] Add an off-heap variant
of the managed memory
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/MemorySegmentSpeedBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/MemorySegmentSpeedBenchmark.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/MemorySegmentSpeedBenchmark.java
new file mode 100644
index 0000000..454c821
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/MemorySegmentSpeedBenchmark.java
@@ -0,0 +1,1633 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory.benchmarks;
+
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.core.memory.HybridMemorySegment;
+import org.apache.flink.core.memory.MemorySegment;
+
+import java.nio.ByteBuffer;
+import java.util.Random;
+
+@SuppressWarnings("ConstantConditions")
+public class MemorySegmentSpeedBenchmark {
+
+ private static final long LONG_VALUE = 0x1234567890abcdefl;
+
+ private static final boolean TEST_CORE_ON_HEAP = true;
+ private static final boolean TEST_CORE_OFF_HEAP = false;
+
+ // we keep this to make sure the JIT does not eliminate certain loops
+ public static long sideEffect = 0L;
+
+
+ public static void main(String[] args) {
+
+ final int SMALL_SEGMENT_SIZE = 32 * 1024;
+ final int LARGE_SEGMENT_SIZE = 1024 * 1024 * 1024;
+
+ final int SMALL_SEGMENTS_ROUNDS = 100000;
+ final int LARGE_SEGMENT_ROUNDS = 10;
+
+ final byte[] largeSegment = new byte[LARGE_SEGMENT_SIZE];
+ final byte[] smallSegment = new byte[SMALL_SEGMENT_SIZE];
+
+ final ByteBuffer largeOffHeap = ByteBuffer.allocateDirect(LARGE_SEGMENT_SIZE);
+ final ByteBuffer smallOffHeap = ByteBuffer.allocateDirect(SMALL_SEGMENT_SIZE);
+
+ System.out.println("testing access of individual bytes");
+
+ testPutBytes(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE, SMALL_SEGMENTS_ROUNDS);
+ testGetBytes(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE, SMALL_SEGMENTS_ROUNDS);
+ testPutBytes(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE, LARGE_SEGMENT_ROUNDS);
+ testGetBytes(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE, LARGE_SEGMENT_ROUNDS);
+
+ System.out.println("testing access of byte arrays");
+
+ testPutByteArrays1024(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 1024, SMALL_SEGMENTS_ROUNDS);
+ testGetByteArrays1024(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 1024, SMALL_SEGMENTS_ROUNDS);
+ testPutByteArrays1024(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 1024, LARGE_SEGMENT_ROUNDS);
+ testGetByteArrays1024(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 1024, LARGE_SEGMENT_ROUNDS);
+
+ System.out.println("testing access of longs");
+
+ testPutLongs(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
+ testGetLongs(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
+ testPutLongs(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
+ testGetLongs(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
+
+// System.out.println("testing access of big endian longs");
+//
+// testPutLongsBigEndian(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
+// testGetLongsBigEndian(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
+// testPutLongsBigEndian(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
+// testGetLongsBigEndian(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
+//
+// System.out.println("testing access of little endian longs");
+//
+// testPutLongsLittleEndian(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
+// testGetLongsLittleEndian(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
+// testPutLongsLittleEndian(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
+// testGetLongsLittleEndian(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
+
+ System.out.println("testing access of ints");
+
+ testPutInts(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 4, SMALL_SEGMENTS_ROUNDS);
+ testGetInts(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 4, SMALL_SEGMENTS_ROUNDS);
+ testPutInts(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 4, LARGE_SEGMENT_ROUNDS);
+ testGetInts(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 4, LARGE_SEGMENT_ROUNDS);
+
+
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // BYTEs
+ // --------------------------------------------------------------------------------------------
+
+ private static void testPutBytes(final byte[] heapMemory, final ByteBuffer offHeapMemory,
+ final int numValues, final int rounds) {
+
+ TestRunner pureHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+ return timePutBytesOnHeap(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+ return timePutBytesHybrid(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+ return timePutBytesHybrid(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timePutBytesAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timePutBytesAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+ return timePutBytesAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner[] tests = {
+ TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+ pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+ };
+
+ long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+ System.out.println(String.format(
+ "Writing %d x %d bytes to %d bytes segment: " +
+ "\n\theap=%,d msecs" +
+ "\n\thybrid-on-heap=%,d msecs" +
+ "\n\thybrid-off-heap=%,d msecs" +
+ "\n\tspecialized heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+ rounds, numValues, heapMemory.length,
+ (results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+ (results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+ }
+
+ private static void testGetBytes(final byte[] heapMemory, final ByteBuffer offHeapMemory,
+ final int numValues, final int rounds) {
+
+ TestRunner pureHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+ return timeGetBytesOnHeap(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+ return timeGetBytesHybrid(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+ return timeGetBytesHybrid(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timeGetBytesAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timeGetBytesAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+ return timeGetBytesAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner[] tests = {
+ TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+ pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+ };
+
+ long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+ System.out.println(String.format(
+ "Reading %d x %d bytes from %d bytes segment: " +
+ "\n\theap=%,d msecs" +
+ "\n\thybrid-on-heap=%,d msecs" +
+ "\n\thybrid-off-heap=%,d msecs" +
+ "\n\tspecialized heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+ rounds, numValues, heapMemory.length,
+ (results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+ (results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+ }
+
+ private static long timePutBytesOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.put(offset, (byte) i);
+ offset++;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timePutBytesOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.put(offset, (byte) i);
+ offset++;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timePutBytesHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.put(offset, (byte) i);
+ offset++;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timePutBytesAbstract(final MemorySegment segment, final int num, final int rounds) {
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.put(offset, (byte) i);
+ offset++;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timeGetBytesOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+ long l = 0;
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ l += segment.get(offset);
+ offset++;
+ }
+ }
+ long end = System.nanoTime();
+ sideEffect += l;
+ return end - start;
+ }
+
+ private static long timeGetBytesOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+ long l = 0;
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ l += segment.get(offset);
+ offset++;
+ }
+ }
+ long end = System.nanoTime();
+ sideEffect += l;
+ return end - start;
+ }
+
+ private static long timeGetBytesHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+ long l = 0;
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ l += segment.get(offset);
+ offset++;
+ }
+ }
+ long end = System.nanoTime();
+ sideEffect += l;
+ return end - start;
+ }
+
+
+ private static long timeGetBytesAbstract(final MemorySegment segment, final int num, final int rounds) {
+ long l = 0;
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ l += segment.get(offset);
+ offset++;
+ }
+ }
+ long end = System.nanoTime();
+ sideEffect += l;
+ return end - start;
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // LONGs
+ // --------------------------------------------------------------------------------------------
+
+ private static void testPutLongs(final byte[] heapMemory, final ByteBuffer offHeapMemory,
+ final int numValues, final int rounds) {
+
+ TestRunner pureHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+ return timePutLongsOnHeap(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+ return timePutLongsHybrid(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+ return timePutLongsHybrid(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timePutLongsAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timePutLongsAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+ return timePutLongsAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner[] tests = {
+ TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+ pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+ };
+
+ long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+ System.out.println(String.format(
+ "Writing %d x %d longs to %d bytes segment: " +
+ "\n\theap=%,d msecs" +
+ "\n\thybrid-on-heap=%,d msecs" +
+ "\n\thybrid-off-heap=%,d msecs" +
+ "\n\tspecialized heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+ rounds, numValues, heapMemory.length,
+ (results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+ (results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+ }
+
+ private static void testGetLongs(final byte[] heapMemory, final ByteBuffer offHeapMemory,
+ final int numValues, final int rounds) {
+
+ TestRunner pureHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+ return timeGetLongsOnHeap(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+ return timeGetLongsHybrid(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+ return timeGetLongsHybrid(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timeGetLongsAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timeGetLongsAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+ return timeGetLongsAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner[] tests = {
+ TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+ pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+ };
+
+ long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+ System.out.println(String.format(
+ "Reading %d x %d longs from %d bytes segment: " +
+ "\n\theap=%,d msecs" +
+ "\n\thybrid-on-heap=%,d msecs" +
+ "\n\thybrid-off-heap=%,d msecs" +
+ "\n\tspecialized heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+ rounds, numValues, heapMemory.length,
+ (results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+ (results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+ }
+
+ private static long timePutLongsOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.putLong(offset, LONG_VALUE);
+ offset += 8;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timePutLongsOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+ // checked segment
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.putLong(offset, LONG_VALUE);
+ offset += 8;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timePutLongsHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+ // checked segment
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.putLong(offset, LONG_VALUE);
+ offset += 8;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timePutLongsAbstract(final MemorySegment segment, final int num, final int rounds) {
+ // checked segment
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.putLong(offset, LONG_VALUE);
+ offset += 8;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timeGetLongsOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+ long l = 0;
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ l += segment.getLong(offset);
+ offset += 8;
+ }
+ }
+ long end = System.nanoTime();
+ sideEffect += l;
+ return end - start;
+ }
+
+ private static long timeGetLongsOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+ long l = 0;
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ l += segment.getLong(offset);
+ offset += 8;
+ }
+ }
+ long end = System.nanoTime();
+ sideEffect += l;
+ return end - start;
+ }
+
+ private static long timeGetLongsHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+ // checked segment
+ long l = 0;
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ l += segment.getLong(offset);
+ offset += 8;
+ }
+ }
+ long end = System.nanoTime();
+ sideEffect += l;
+ return end - start;
+ }
+
+ private static long timeGetLongsAbstract(final MemorySegment segment, final int num, final int rounds) {
+ // checked segment
+ long l = 0;
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ l += segment.getLong(offset);
+ offset += 8;
+ }
+ }
+ long end = System.nanoTime();
+ sideEffect += l;
+ return end - start;
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // INTs
+ // --------------------------------------------------------------------------------------------
+
+ private static void testPutInts(final byte[] heapMemory, final ByteBuffer offHeapMemory,
+ final int numValues, final int rounds) {
+
+ TestRunner pureHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+ return timePutIntsOnHeap(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+ return timePutIntsHybrid(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+ return timePutIntsHybrid(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timePutIntsAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timePutIntsAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+ return timePutIntsAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner[] tests = {
+ TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+ pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+ };
+
+ long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+ System.out.println(String.format(
+ "Writing %d x %d ints to %d bytes segment: " +
+ "\n\theap=%,d msecs" +
+ "\n\thybrid-on-heap=%,d msecs" +
+ "\n\thybrid-off-heap=%,d msecs" +
+ "\n\tspecialized heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+ rounds, numValues, heapMemory.length,
+ (results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+ (results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+ }
+
+ private static void testGetInts(final byte[] heapMemory, final ByteBuffer offHeapMemory,
+ final int numValues, final int rounds) {
+
+ TestRunner pureHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+ return timeGetIntsOnHeap(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+ return timeGetIntsHybrid(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+ return timeGetIntsHybrid(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timeGetIntsAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timeGetIntsAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+ return timeGetIntsAbstract(seg, numValues, rounds);
+ }
+ };
+
+ TestRunner[] tests = {
+ TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+ pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+ };
+
+ long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+ System.out.println(String.format(
+ "Reading %d x %d ints from %d bytes segment: " +
+ "\n\theap=%,d msecs" +
+ "\n\thybrid-on-heap=%,d msecs" +
+ "\n\thybrid-off-heap=%,d msecs" +
+ "\n\tspecialized heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+ rounds, numValues, heapMemory.length,
+ (results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+ (results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+ }
+
+ private static long timePutIntsOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.putInt(offset, i);
+ offset += 4;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timePutIntsOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+ // checked segment
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.putInt(offset, i);
+ offset += 4;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timePutIntsHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+ // checked segment
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.putInt(offset, i);
+ offset += 4;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timePutIntsAbstract(final MemorySegment segment, final int num, final int rounds) {
+ // checked segment
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.putInt(offset, i);
+ offset += 4;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timeGetIntsOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+ int l = 0;
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ l += segment.getInt(offset);
+ offset += 4;
+ }
+ }
+ long end = System.nanoTime();
+ sideEffect += l;
+ return end - start;
+ }
+
+ private static long timeGetIntsOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+ int l = 0;
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ l += segment.getInt(offset);
+ offset += 4;
+ }
+ }
+ long end = System.nanoTime();
+ sideEffect += l;
+ return end - start;
+ }
+
+ private static long timeGetIntsHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+ int l = 0;
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ l += segment.getInt(offset);
+ offset += 4;
+ }
+ }
+ long end = System.nanoTime();
+ sideEffect += l;
+ return end - start;
+ }
+
+ private static long timeGetIntsAbstract(final MemorySegment segment, final int num, final int rounds) {
+ int l = 0;
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ l += segment.getInt(offset);
+ offset += 4;
+ }
+ }
+ long end = System.nanoTime();
+ sideEffect += l;
+ return end - start;
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // BYTE ARRAYs
+ // --------------------------------------------------------------------------------------------
+
+ private static void testPutByteArrays1024(final byte[] heapMemory, final ByteBuffer offHeapMemory,
+ final int numValues, final int rounds) {
+
+ final byte[] sourceArray = new byte[1024];
+ for (int i = 0; i < sourceArray.length; i++) {
+ sourceArray[i] = (byte) i;
+ }
+
+ TestRunner pureHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+ return timePutByteArrayOnHeap(seg, sourceArray, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+ return timePutByteArrayHybrid(seg, sourceArray, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+ return timePutByteArrayHybrid(seg, sourceArray, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timePutByteArrayAbstract(seg, sourceArray, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timePutByteArrayAbstract(seg, sourceArray, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+ return timePutByteArrayAbstract(seg, sourceArray, numValues, rounds);
+ }
+ };
+
+ TestRunner[] tests = {
+ TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+ pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+ };
+
+ long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+ System.out.println(String.format(
+ "Writing %d x %d byte[1024] to %d bytes segment: " +
+ "\n\theap=%,d msecs" +
+ "\n\thybrid-on-heap=%,d msecs" +
+ "\n\thybrid-off-heap=%,d msecs" +
+ "\n\tspecialized heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+ rounds, numValues, heapMemory.length,
+ (results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+ (results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+ }
+
+ private static void testGetByteArrays1024(final byte[] heapMemory, final ByteBuffer offHeapMemory,
+ final int numValues, final int rounds) {
+
+ final byte[] targetArray = new byte[1024];
+
+ TestRunner pureHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+ return timeGetByteArrayOnHeap(seg, targetArray, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+ return timeGetByteArrayHybrid(seg, targetArray, numValues, rounds);
+ }
+ };
+
+ TestRunner pureHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+ return timeGetByteArrayHybrid(seg, targetArray, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timeGetByteArrayAbstract(seg, targetArray, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOnHeap(heapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+ return timeGetByteArrayAbstract(seg, targetArray, numValues, rounds);
+ }
+ };
+
+ TestRunner coreHybridOffHeapRunner = new TestRunner() {
+ @Override
+ public long runTest() {
+ fillOffHeap(offHeapMemory, (byte) 0);
+ MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+ return timeGetByteArrayAbstract(seg, targetArray, numValues, rounds);
+ }
+ };
+
+ TestRunner[] tests = {
+ TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+ TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+ pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+ };
+
+ long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+ System.out.println(String.format(
+ "Reading %d x %d byte[1024] from %d bytes segment: " +
+ "\n\theap=%,d msecs" +
+ "\n\thybrid-on-heap=%,d msecs" +
+ "\n\thybrid-off-heap=%,d msecs" +
+ "\n\tspecialized heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-heap=%,d msecs, " +
+ "\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+ rounds, numValues, heapMemory.length,
+ (results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+ (results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+ }
+
+ private static long timePutByteArrayOnHeap(final PureHeapMemorySegment segment, final byte[] source, final int num, final int rounds) {
+ final int len = source.length;
+
+ // checked segment
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.put(offset, source, 0, len);
+ offset += len;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timePutByteArrayOffHeap(final PureOffHeapMemorySegment segment, final byte[] source, final int num, final int rounds) {
+ final int len = source.length;
+
+ // checked segment
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.put(offset, source, 0, len);
+ offset += len;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timePutByteArrayHybrid(final PureHybridMemorySegment segment, final byte[] source, final int num, final int rounds) {
+ final int len = source.length;
+
+ // checked segment
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.put(offset, source, 0, len);
+ offset += len;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timePutByteArrayAbstract(final MemorySegment segment, final byte[] source, final int num, final int rounds) {
+ final int len = source.length;
+
+ // checked segment
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.put(offset, source, 0, len);
+ offset += len;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timeGetByteArrayOnHeap(final PureHeapMemorySegment segment, final byte[] target, final int num, final int rounds) {
+ final int len = target.length;
+
+ // checked segment
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.get(offset, target, 0, len);
+ offset += len;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timeGetByteArrayOffHeap(final PureOffHeapMemorySegment segment, final byte[] target, final int num, final int rounds) {
+ final int len = target.length;
+
+ // checked segment
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.get(offset, target, 0, len);
+ offset += len;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timeGetByteArrayHybrid(final PureHybridMemorySegment segment, final byte[] target, final int num, final int rounds) {
+ final int len = target.length;
+
+ // checked segment
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.get(offset, target, 0, len);
+ offset += len;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+ private static long timeGetByteArrayAbstract(final MemorySegment segment, final byte[] target, final int num, final int rounds) {
+ final int len = target.length;
+
+ // checked segment
+ long start = System.nanoTime();
+ for (int round = 0; round < rounds; round++) {
+ int offset = 0;
+ for (int i = 0; i < num; i++) {
+ segment.get(offset, target, 0, len);
+ offset += len;
+ }
+ }
+ long end = System.nanoTime();
+ return end - start;
+ }
+
+// // --------------------------------------------------------------------------------------------
+// // LONG BIG ENDIAN
+// // --------------------------------------------------------------------------------------------
+//
+// private static void testPutLongsBigEndian(byte[] heapMemory, ByteBuffer offHeapMemory, int numValues, int rounds) {
+// // test the pure heap memory
+// fillOnHeap(heapMemory, (byte) 0);
+// PureHeapMemorySegment heapMemorySegment = new PureHeapMemorySegment(heapMemory);
+// long elapsedOnHeap = timePutLongsBigEndianOnHeap(heapMemorySegment, numValues, rounds);
+//
+// // test the pure off-heap memory
+// fillOffHeap(offHeapMemory, (byte) 0);
+// PureOffHeapMemorySegment offHeapMemorySegment = new PureOffHeapMemorySegment(offHeapMemory);
+// long elapsedOffHeap = timePutLongsBigEndianOffHeap(offHeapMemorySegment, numValues, rounds);
+//
+// // test hybrid on heap
+// fillOnHeap(heapMemory, (byte) 0);
+// PureHybridMemorySegment hybridOnHeap = new PureHybridMemorySegment(heapMemory);
+// long elapsedHybridOnHeap = timePutLongsBigEndianHybrid(hybridOnHeap, numValues, rounds);
+//
+// // test hybrid off heap
+// fillOffHeap(offHeapMemory, (byte) 0);
+// PureHybridMemorySegment hybridOffeap = new PureHybridMemorySegment(offHeapMemory);
+// long elapsedHybridOffHeap = timePutLongsBigEndianHybrid(hybridOffeap, numValues, rounds);
+//
+// System.out.println(String.format(
+// "Writing %d x %d big-endian longs to %d bytes segment: " +
+// "heap=%,d msecs, " +
+// "off-heap=%,d msecs, " +
+// "hybrid-on-heap=%,d msecs, " +
+// "hybrid-off-heap=%,d msecs",
+// rounds, numValues, heapMemory.length,
+// (elapsedOnHeap / 1000000), (elapsedOffHeap / 1000000),
+// (elapsedHybridOnHeap / 1000000), (elapsedHybridOffHeap / 1000000)));
+// }
+//
+// private static void testGetLongsBigEndian(byte[] heapMemory, ByteBuffer offHeapMemory, int numValues, int rounds) {
+// // test the pure heap memory
+// fillOnHeap(heapMemory, (byte) 0);
+// PureHeapMemorySegment heapMemorySegment = new PureHeapMemorySegment(heapMemory);
+// long elapsedOnHeap = timeGetLongsBigEndianOnHeap(heapMemorySegment, numValues, rounds);
+//
+// // test the pure off-heap memory
+// fillOffHeap(offHeapMemory, (byte) 0);
+// PureOffHeapMemorySegment offHeapMemorySegment = new PureOffHeapMemorySegment(offHeapMemory);
+// long elapsedOffHeap = timeGetLongsBigEndianOffHeap(offHeapMemorySegment, numValues, rounds);
+//
+// // test hybrid on heap
+// fillOnHeap(heapMemory, (byte) 0);
+// PureHybridMemorySegment hybridOnHeap = new PureHybridMemorySegment(heapMemory);
+// long elapsedHybridOnHeap = timeGetLongsBigEndianHybrid(hybridOnHeap, numValues, rounds);
+//
+// // test hybrid off heap
+// fillOffHeap(offHeapMemory, (byte) 0);
+// PureHybridMemorySegment hybridOffeap = new PureHybridMemorySegment(offHeapMemory);
+// long elapsedHybridOffHeap = timeGetLongsBigEndianHybrid(hybridOffeap, numValues, rounds);
+//
+// System.out.println(String.format(
+// "Reading %d x %d big-endian longs from %d bytes segment: " +
+// "heap=%,d msecs, " +
+// "off-heap=%,d msecs, " +
+// "hybrid-on-heap=%,d msecs, " +
+// "hybrid-off-heap=%,d msecs",
+// rounds, numValues, heapMemory.length,
+// (elapsedOnHeap / 1000000), (elapsedOffHeap / 1000000),
+// (elapsedHybridOnHeap / 1000000), (elapsedHybridOffHeap / 1000000)));
+// }
+//
+// private static long timePutLongsBigEndianOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+// long start = System.nanoTime();
+// for (int round = 0; round < rounds; round++) {
+// int offset = 0;
+// for (int i = 0; i < num; i++) {
+// segment.putLongBigEndian(offset, LONG_VALUE);
+// offset += 8;
+// }
+// }
+// long end = System.nanoTime();
+// return end - start;
+// }
+//
+// private static long timePutLongsBigEndianOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+// // checked segment
+// long start = System.nanoTime();
+// for (int round = 0; round < rounds; round++) {
+// int offset = 0;
+// for (int i = 0; i < num; i++) {
+// segment.putLongBigEndian(offset, LONG_VALUE);
+// offset += 8;
+// }
+// }
+// long end = System.nanoTime();
+// return end - start;
+// }
+//
+// private static long timePutLongsBigEndianHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+// // checked segment
+// long start = System.nanoTime();
+// for (int round = 0; round < rounds; round++) {
+// int offset = 0;
+// for (int i = 0; i < num; i++) {
+// segment.putLongBigEndian(offset, LONG_VALUE);
+// offset += 8;
+// }
+// }
+// long end = System.nanoTime();
+// return end - start;
+// }
+//
+// private static long timeGetLongsBigEndianOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+// long l = 0;
+// long start = System.nanoTime();
+// for (int round = 0; round < rounds; round++) {
+// int offset = 0;
+// for (int i = 0; i < num; i++) {
+// l += segment.getLongBigEndian(offset);
+// offset += 8;
+// }
+// }
+// long end = System.nanoTime();
+// sideEffect += l;
+// return end - start;
+// }
+//
+// private static long timeGetLongsBigEndianOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+// long l = 0;
+// long start = System.nanoTime();
+// for (int round = 0; round < rounds; round++) {
+// int offset = 0;
+// for (int i = 0; i < num; i++) {
+// l += segment.getLongBigEndian(offset);
+// offset += 8;
+// }
+// }
+// long end = System.nanoTime();
+// sideEffect += l;
+// return end - start;
+// }
+//
+// private static long timeGetLongsBigEndianHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+// // checked segment
+// long l = 0;
+// long start = System.nanoTime();
+// for (int round = 0; round < rounds; round++) {
+// int offset = 0;
+// for (int i = 0; i < num; i++) {
+// l += segment.getLongBigEndian(offset);
+// offset += 8;
+// }
+// }
+// long end = System.nanoTime();
+// sideEffect += l;
+// return end - start;
+// }
+//
+// // --------------------------------------------------------------------------------------------
+// // LONG LITTLE ENDIAN
+// // --------------------------------------------------------------------------------------------
+//
+// private static void testPutLongsLittleEndian(byte[] heapMemory, ByteBuffer offHeapMemory, int numValues, int rounds) {
+// // test the pure heap memory
+// fillOnHeap(heapMemory, (byte) 0);
+// PureHeapMemorySegment heapMemorySegment = new PureHeapMemorySegment(heapMemory);
+// long elapsedOnHeap = timePutLongsLittleEndianOnHeap(heapMemorySegment, numValues, rounds);
+//
+// // test the pure off-heap memory
+// fillOffHeap(offHeapMemory, (byte) 0);
+// PureOffHeapMemorySegment offHeapMemorySegment = new PureOffHeapMemorySegment(offHeapMemory);
+// long elapsedOffHeap = timePutLongsLittleEndianOffHeap(offHeapMemorySegment, numValues, rounds);
+//
+// // test hybrid on heap
+// fillOnHeap(heapMemory, (byte) 0);
+// PureHybridMemorySegment hybridOnHeap = new PureHybridMemorySegment(heapMemory);
+// long elapsedHybridOnHeap = timePutLongsLittleEndianHybrid(hybridOnHeap, numValues, rounds);
+//
+// // test hybrid off heap
+// fillOffHeap(offHeapMemory, (byte) 0);
+// PureHybridMemorySegment hybridOffeap = new PureHybridMemorySegment(offHeapMemory);
+// long elapsedHybridOffHeap = timePutLongsLittleEndianHybrid(hybridOffeap, numValues, rounds);
+//
+// System.out.println(String.format(
+// "Writing %d x %d little-endian longs to %d bytes segment: " +
+// "heap=%,d msecs, " +
+// "off-heap=%,d msecs, " +
+// "hybrid-on-heap=%,d msecs, " +
+// "hybrid-off-heap=%,d msecs",
+// rounds, numValues, heapMemory.length,
+// (elapsedOnHeap / 1000000), (elapsedOffHeap / 1000000),
+// (elapsedHybridOnHeap / 1000000), (elapsedHybridOffHeap / 1000000)));
+// }
+//
+// private static void testGetLongsLittleEndian(byte[] heapMemory, ByteBuffer offHeapMemory, int numValues, int rounds) {
+// // test the pure heap memory
+// fillOnHeap(heapMemory, (byte) 0);
+// PureHeapMemorySegment heapMemorySegment = new PureHeapMemorySegment(heapMemory);
+// long elapsedOnHeap = timeGetLongsLittleEndianOnHeap(heapMemorySegment, numValues, rounds);
+//
+// // test the pure off-heap memory
+// fillOffHeap(offHeapMemory, (byte) 0);
+// PureOffHeapMemorySegment offHeapMemorySegment = new PureOffHeapMemorySegment(offHeapMemory);
+// long elapsedOffHeap = timeGetLongsLittleEndianOffHeap(offHeapMemorySegment, numValues, rounds);
+//
+// // test hybrid on heap
+// fillOnHeap(heapMemory, (byte) 0);
+// PureHybridMemorySegment hybridOnHeap = new PureHybridMemorySegment(heapMemory);
+// long elapsedHybridOnHeap = timeGetLongsLittleEndianHybrid(hybridOnHeap, numValues, rounds);
+//
+// // test hybrid off heap
+// fillOffHeap(offHeapMemory, (byte) 0);
+// PureHybridMemorySegment hybridOffeap = new PureHybridMemorySegment(offHeapMemory);
+// long elapsedHybridOffHeap = timeGetLongsLittleEndianHybrid(hybridOffeap, numValues, rounds);
+//
+// System.out.println(String.format(
+// "Reading %d x %d little-endian longs from %d bytes segment: " +
+// "heap=%,d msecs, " +
+// "off-heap=%,d msecs, " +
+// "hybrid-on-heap=%,d msecs, " +
+// "hybrid-off-heap=%,d msecs",
+// rounds, numValues, heapMemory.length,
+// (elapsedOnHeap / 1000000), (elapsedOffHeap / 1000000),
+// (elapsedHybridOnHeap / 1000000), (elapsedHybridOffHeap / 1000000)));
+// }
+//
+// private static long timePutLongsLittleEndianOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+// long start = System.nanoTime();
+// for (int round = 0; round < rounds; round++) {
+// int offset = 0;
+// for (int i = 0; i < num; i++) {
+// segment.putLongLittleEndian(offset, LONG_VALUE);
+// offset += 8;
+// }
+// }
+// long end = System.nanoTime();
+// return end - start;
+// }
+//
+// private static long timePutLongsLittleEndianOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+// // checked segment
+// long start = System.nanoTime();
+// for (int round = 0; round < rounds; round++) {
+// int offset = 0;
+// for (int i = 0; i < num; i++) {
+// segment.putLongLittleEndian(offset, LONG_VALUE);
+// offset += 8;
+// }
+// }
+// long end = System.nanoTime();
+// return end - start;
+// }
+//
+// private static long timePutLongsLittleEndianHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+// // checked segment
+// long start = System.nanoTime();
+// for (int round = 0; round < rounds; round++) {
+// int offset = 0;
+// for (int i = 0; i < num; i++) {
+// segment.putLongLittleEndian(offset, LONG_VALUE);
+// offset += 8;
+// }
+// }
+// long end = System.nanoTime();
+// return end - start;
+// }
+//
+// private static long timeGetLongsLittleEndianOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+// long l = 0;
+// long start = System.nanoTime();
+// for (int round = 0; round < rounds; round++) {
+// int offset = 0;
+// for (int i = 0; i < num; i++) {
+// l += segment.getLongLittleEndian(offset);
+// offset += 8;
+// }
+// }
+// long end = System.nanoTime();
+// sideEffect += l;
+// return end - start;
+// }
+//
+// private static long timeGetLongsLittleEndianOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+// long l = 0;
+// long start = System.nanoTime();
+// for (int round = 0; round < rounds; round++) {
+// int offset = 0;
+// for (int i = 0; i < num; i++) {
+// l += segment.getLongLittleEndian(offset);
+// offset += 8;
+// }
+// }
+// long end = System.nanoTime();
+// sideEffect += l;
+// return end - start;
+// }
+//
+// private static long timeGetLongsLittleEndianHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+// // checked segment
+// long l = 0;
+// long start = System.nanoTime();
+// for (int round = 0; round < rounds; round++) {
+// int offset = 0;
+// for (int i = 0; i < num; i++) {
+// l += segment.getLongLittleEndian(offset);
+// offset += 8;
+// }
+// }
+// long end = System.nanoTime();
+// sideEffect += l;
+// return end - start;
+// }
+
+ // ------------------------------------------------------------------------
+ // Utilities
+ // ------------------------------------------------------------------------
+
+ private static void fillOnHeap(byte[] buffer, byte data) {
+ for (int i = 0; i < buffer.length; i++) {
+ buffer[i] = data;
+ }
+ }
+
+ private static void fillOffHeap(ByteBuffer buffer, byte data) {
+ final int len = buffer.capacity();
+ for (int i = 0; i < len; i++) {
+ buffer.put(i, data);
+ }
+ }
+
+ private static long[] runTestsInRandomOrder(TestRunner[] runners, Random rnd, int numRuns, boolean printMeasures) {
+ if (numRuns < 3) {
+ throw new IllegalArgumentException("must do at least three runs");
+ }
+
+ // we run all runners in random order, to account for the JIT effects that specialize methods
+ // The observation is that either earlier tests suffer from performance because the JIT needs to kick
+ // in first, or that later tests suffer from performance, because the HIT optimized for the other case already
+
+ long[][] measures = new long[runners.length][];
+ for (int i = 0; i < measures.length; i++) {
+ measures[i] = new long[numRuns];
+ }
+
+ for (int test = 0; test < numRuns; test++) {
+ System.out.println("Round " + (test+1) + '/' + numRuns);
+
+ // pick an order for the tests
+ int[] order = new int[runners.length];
+ for (int i = 0; i < order.length; i++) {
+ order[i] = i;
+ }
+ for (int i = order.length; i > 1; i--) {
+ int pos1 = i-1;
+ int pos2 = rnd.nextInt(i);
+ int tmp = order[pos1];
+ order[pos1] = order[pos2];
+ order[pos2] = tmp;
+ }
+
+ // run tests
+ for (int pos : order) {
+ TestRunner next = runners[pos];
+ measures[pos][test] = next != null ? next.runTest() : 0L;
+ }
+ }
+
+ if (printMeasures) {
+ for (long[] series : measures) {
+ StringBuilder bld = new StringBuilder();
+ for (long measure : series) {
+ bld.append(String.format("%,d", (measure / 1000000))).append(" | ");
+ }
+ System.out.println(bld.toString());
+ }
+ }
+
+ // aggregate the measures
+ long[] results = new long[runners.length];
+
+ for (int i = 0; i < runners.length; i++) {
+ // cancel out the min and max
+ long max = Long.MIN_VALUE;
+ long min = Long.MAX_VALUE;
+
+ for (long val : measures[i]) {
+ max = Math.max(max, val);
+ min = Math.min(min, val);
+ }
+
+ long total = 0L;
+ for (long val : measures[i]) {
+ if (val != max && val != min) {
+ total += val;
+ }
+ }
+
+ results[i] = total / (numRuns - 2);
+ }
+
+ return results;
+ }
+
+
+
+ private static interface TestRunner {
+
+ long runTest();
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegment.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegment.java
new file mode 100644
index 0000000..e247eed
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegment.java
@@ -0,0 +1,466 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory.benchmarks;
+
+import org.apache.flink.core.memory.MemoryUtils;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+public final class PureHeapMemorySegment {
+
+ /** Constant that flags the byte order. Because this is a boolean constant,
+ * the JIT compiler can use this well to aggressively eliminate the non-applicable code paths */
+ private static final boolean LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+ /** The array in which the data is stored. */
+ private byte[] memory;
+
+ /** Wrapper for I/O requests. */
+ private ByteBuffer wrapper;
+
+ /** The size, stored extra, because we may clear the reference to the byte array */
+ private final int size;
+
+ // -------------------------------------------------------------------------
+ // Constructors
+ // -------------------------------------------------------------------------
+
+ /**
+ * Creates a new memory segment that represents the data in the given byte array.
+ *
+ * @param memory The byte array that holds the data.
+ */
+ public PureHeapMemorySegment(byte[] memory) {
+ this.memory = memory;
+ this.size = memory.length;
+ }
+
+ // -------------------------------------------------------------------------
+ // Direct Memory Segment Specifics
+ // -------------------------------------------------------------------------
+
+ /**
+ * Gets the byte array that backs this memory segment.
+ *
+ * @return The byte array that backs this memory segment.
+ */
+ public byte[] getArray() {
+ return this.memory;
+ }
+
+ // -------------------------------------------------------------------------
+ // MemorySegment Accessors
+ // -------------------------------------------------------------------------
+
+ public final boolean isFreed() {
+ return this.memory == null;
+ }
+
+ public final void free() {
+ this.wrapper = null;
+ this.memory = null;
+ }
+
+ public final int size() {
+ return this.size;
+ }
+
+ public final ByteBuffer wrap(int offset, int length) {
+ if (offset > this.memory.length || offset > this.memory.length - length) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ if (this.wrapper == null) {
+ this.wrapper = ByteBuffer.wrap(this.memory, offset, length);
+ }
+ else {
+ this.wrapper.limit(offset + length);
+ this.wrapper.position(offset);
+ }
+
+ return this.wrapper;
+ }
+
+ // ------------------------------------------------------------------------
+ // Random Access get() and put() methods
+ // ------------------------------------------------------------------------
+
+ public final byte get(int index) {
+ return this.memory[index];
+ }
+
+ public final void put(int index, byte b) {
+ this.memory[index] = b;
+ }
+
+ public final void get(int index, byte[] dst) {
+ get(index, dst, 0, dst.length);
+ }
+
+ public final void put(int index, byte[] src) {
+ put(index, src, 0, src.length);
+ }
+
+ public final void get(int index, byte[] dst, int offset, int length) {
+ // system arraycopy does the boundary checks anyways, no need to check extra
+ System.arraycopy(this.memory, index, dst, offset, length);
+ }
+
+ public final void put(int index, byte[] src, int offset, int length) {
+ // system arraycopy does the boundary checks anyways, no need to check extra
+ System.arraycopy(src, offset, this.memory, index, length);
+ }
+
+ public final boolean getBoolean(int index) {
+ return this.memory[index] != 0;
+ }
+
+ public final void putBoolean(int index, boolean value) {
+ this.memory[index] = (byte) (value ? 1 : 0);
+ }
+
+ @SuppressWarnings("restriction")
+ public final char getChar(int index) {
+ if (index >= 0 && index <= this.memory.length - 2) {
+ return UNSAFE.getChar(this.memory, BASE_OFFSET + index);
+ } else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final char getCharLittleEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return getChar(index);
+ } else {
+ return Character.reverseBytes(getChar(index));
+ }
+ }
+
+ public final char getCharBigEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return Character.reverseBytes(getChar(index));
+ } else {
+ return getChar(index);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void putChar(int index, char value) {
+ if (index >= 0 && index <= this.memory.length - 2) {
+ UNSAFE.putChar(this.memory, BASE_OFFSET + index, value);
+ } else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final void putCharLittleEndian(int index, char value) {
+ if (LITTLE_ENDIAN) {
+ putChar(index, value);
+ } else {
+ putChar(index, Character.reverseBytes(value));
+ }
+ }
+
+ public final void putCharBigEndian(int index, char value) {
+ if (LITTLE_ENDIAN) {
+ putChar(index, Character.reverseBytes(value));
+ } else {
+ putChar(index, value);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final short getShort(int index) {
+ if (index >= 0 && index <= this.memory.length - 2) {
+ return UNSAFE.getShort(this.memory, BASE_OFFSET + index);
+ } else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final short getShortLittleEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return getShort(index);
+ } else {
+ return Short.reverseBytes(getShort(index));
+ }
+ }
+
+ public final short getShortBigEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return Short.reverseBytes(getShort(index));
+ } else {
+ return getShort(index);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void putShort(int index, short value) {
+ if (index >= 0 && index <= this.memory.length - 2) {
+ UNSAFE.putShort(this.memory, BASE_OFFSET + index, value);
+ } else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final void putShortLittleEndian(int index, short value) {
+ if (LITTLE_ENDIAN) {
+ putShort(index, value);
+ } else {
+ putShort(index, Short.reverseBytes(value));
+ }
+ }
+
+ public final void putShortBigEndian(int index, short value) {
+ if (LITTLE_ENDIAN) {
+ putShort(index, Short.reverseBytes(value));
+ } else {
+ putShort(index, value);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final int getInt(int index) {
+ if (index >= 0 && index <= this.memory.length - 4) {
+ return UNSAFE.getInt(this.memory, BASE_OFFSET + index);
+ } else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final int getIntLittleEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return getInt(index);
+ } else {
+ return Integer.reverseBytes(getInt(index));
+ }
+ }
+
+ public final int getIntBigEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return Integer.reverseBytes(getInt(index));
+ } else {
+ return getInt(index);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void putInt(int index, int value) {
+ if (index >= 0 && index <= this.memory.length - 4) {
+ UNSAFE.putInt(this.memory, BASE_OFFSET + index, value);
+ } else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final void putIntLittleEndian(int index, int value) {
+ if (LITTLE_ENDIAN) {
+ putInt(index, value);
+ } else {
+ putInt(index, Integer.reverseBytes(value));
+ }
+ }
+
+ public final void putIntBigEndian(int index, int value) {
+ if (LITTLE_ENDIAN) {
+ putInt(index, Integer.reverseBytes(value));
+ } else {
+ putInt(index, value);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final long getLong(int index) {
+ if (index >= 0 && index <= this.memory.length - 8) {
+ return UNSAFE.getLong(this.memory, BASE_OFFSET + index);
+ } else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final long getLongLittleEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return getLong(index);
+ } else {
+ return Long.reverseBytes(getLong(index));
+ }
+ }
+
+ public final long getLongBigEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return Long.reverseBytes(getLong(index));
+ } else {
+ return getLong(index);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void putLong(int index, long value) {
+ if (index >= 0 && index <= this.memory.length - 8) {
+ UNSAFE.putLong(this.memory, BASE_OFFSET + index, value);
+ } else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final void putLongLittleEndian(int index, long value) {
+ if (LITTLE_ENDIAN) {
+ putLong(index, value);
+ } else {
+ putLong(index, Long.reverseBytes(value));
+ }
+ }
+
+ public final void putLongBigEndian(int index, long value) {
+ if (LITTLE_ENDIAN) {
+ putLong(index, Long.reverseBytes(value));
+ } else {
+ putLong(index, value);
+ }
+ }
+
+ public final float getFloat(int index) {
+ return Float.intBitsToFloat(getInt(index));
+ }
+
+ public final float getFloatLittleEndian(int index) {
+ return Float.intBitsToFloat(getIntLittleEndian(index));
+ }
+
+ public final float getFloatBigEndian(int index) {
+ return Float.intBitsToFloat(getIntBigEndian(index));
+ }
+
+ public final void putFloat(int index, float value) {
+ putInt(index, Float.floatToRawIntBits(value));
+ }
+
+ public final void putFloatLittleEndian(int index, float value) {
+ putIntLittleEndian(index, Float.floatToRawIntBits(value));
+ }
+
+ public final void putFloatBigEndian(int index, float value) {
+ putIntBigEndian(index, Float.floatToRawIntBits(value));
+ }
+
+ public final double getDouble(int index) {
+ return Double.longBitsToDouble(getLong(index));
+ }
+
+ public final double getDoubleLittleEndian(int index) {
+ return Double.longBitsToDouble(getLongLittleEndian(index));
+ }
+
+ public final double getDoubleBigEndian(int index) {
+ return Double.longBitsToDouble(getLongBigEndian(index));
+ }
+
+ public final void putDouble(int index, double value) {
+ putLong(index, Double.doubleToRawLongBits(value));
+ }
+
+ public final void putDoubleLittleEndian(int index, double value) {
+ putLongLittleEndian(index, Double.doubleToRawLongBits(value));
+ }
+
+ public final void putDoubleBigEndian(int index, double value) {
+ putLongBigEndian(index, Double.doubleToRawLongBits(value));
+ }
+
+ // -------------------------------------------------------------------------
+ // Bulk Read and Write Methods
+ // -------------------------------------------------------------------------
+
+ public final void get(DataOutput out, int offset, int length) throws IOException {
+ out.write(this.memory, offset, length);
+ }
+
+ public final void put(DataInput in, int offset, int length) throws IOException {
+ in.readFully(this.memory, offset, length);
+ }
+
+ public final void get(int offset, ByteBuffer target, int numBytes) {
+ // ByteBuffer performs the boundary checks
+ target.put(this.memory, offset, numBytes);
+ }
+
+ public final void put(int offset, ByteBuffer source, int numBytes) {
+ // ByteBuffer performs the boundary checks
+ source.get(this.memory, offset, numBytes);
+ }
+
+ public final void copyTo(int offset, PureHeapMemorySegment target, int targetOffset, int numBytes) {
+ // system arraycopy does the boundary checks anyways, no need to check extra
+ System.arraycopy(this.memory, offset, target.memory, targetOffset, numBytes);
+ }
+
+ // -------------------------------------------------------------------------
+ // Comparisons & Swapping
+ // -------------------------------------------------------------------------
+
+ public final int compare(PureHeapMemorySegment seg2, int offset1, int offset2, int len) {
+ final byte[] b2 = seg2.memory;
+ final byte[] b1 = this.memory;
+
+ int val = 0;
+ for (int pos = 0; pos < len && (val = (b1[offset1 + pos] & 0xff) - (b2[offset2 + pos] & 0xff)) == 0; pos++);
+ return val;
+ }
+
+ public final void swapBytes(PureHeapMemorySegment seg2, int offset1, int offset2, int len) {
+ // swap by bytes (chunks of 8 first, then single bytes)
+ while (len >= 8) {
+ long tmp = this.getLong(offset1);
+ this.putLong(offset1, seg2.getLong(offset2));
+ seg2.putLong(offset2, tmp);
+ offset1 += 8;
+ offset2 += 8;
+ len -= 8;
+ }
+ while (len > 0) {
+ byte tmp = this.get(offset1);
+ this.put(offset1, seg2.get(offset2));
+ seg2.put(offset2, tmp);
+ offset1++;
+ offset2++;
+ len--;
+ }
+ }
+
+ public final void swapBytes(byte[] auxBuffer, PureHeapMemorySegment seg2, int offset1, int offset2, int len) {
+ byte[] otherMem = seg2.memory;
+ System.arraycopy(this.memory, offset1, auxBuffer, 0, len);
+ System.arraycopy(otherMem, offset2, this.memory, offset1, len);
+ System.arraycopy(auxBuffer, 0, otherMem, offset2, len);
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Utilities for native memory accesses and checks
+ // --------------------------------------------------------------------------------------------
+
+ @SuppressWarnings("restriction")
+ private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+
+ @SuppressWarnings("restriction")
+ private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegmentOutView.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegmentOutView.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegmentOutView.java
new file mode 100644
index 0000000..1e3b89e
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegmentOutView.java
@@ -0,0 +1,359 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory.benchmarks;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.util.List;
+
+public final class PureHeapMemorySegmentOutView implements DataOutputView {
+
+ private PureHeapMemorySegment currentSegment; // the current memory segment to write to
+
+ private int positionInSegment; // the offset in the current segment
+
+ private final int segmentSize; // the size of the memory segments
+
+ private final List<PureHeapMemorySegment> memorySource;
+
+ private final List<PureHeapMemorySegment> fullSegments;
+
+
+ private byte[] utfBuffer; // the reusable array for UTF encodings
+
+
+ public PureHeapMemorySegmentOutView(List<PureHeapMemorySegment> emptySegments,
+ List<PureHeapMemorySegment> fullSegmentTarget, int segmentSize) {
+ this.segmentSize = segmentSize;
+ this.currentSegment = emptySegments.remove(emptySegments.size() - 1);
+
+ this.memorySource = emptySegments;
+ this.fullSegments = fullSegmentTarget;
+ this.fullSegments.add(getCurrentSegment());
+ }
+
+
+ public void reset() {
+ if (this.fullSegments.size() != 0) {
+ throw new IllegalStateException("The target list still contains memory segments.");
+ }
+
+ clear();
+ try {
+ advance();
+ }
+ catch (IOException ioex) {
+ throw new RuntimeException("Error getting first segment for record collector.", ioex);
+ }
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Page Management
+ // --------------------------------------------------------------------------------------------
+
+ public PureHeapMemorySegment nextSegment(PureHeapMemorySegment current, int positionInCurrent) throws EOFException {
+ int size = this.memorySource.size();
+ if (size > 0) {
+ final PureHeapMemorySegment next = this.memorySource.remove(size - 1);
+ this.fullSegments.add(next);
+ return next;
+ } else {
+ throw new EOFException();
+ }
+ }
+
+ public PureHeapMemorySegment getCurrentSegment() {
+ return this.currentSegment;
+ }
+
+ public int getCurrentPositionInSegment() {
+ return this.positionInSegment;
+ }
+
+ public int getSegmentSize() {
+ return this.segmentSize;
+ }
+
+ protected void advance() throws IOException {
+ this.currentSegment = nextSegment(this.currentSegment, this.positionInSegment);
+ this.positionInSegment = 0;
+ }
+
+ protected void seekOutput(PureHeapMemorySegment seg, int position) {
+ this.currentSegment = seg;
+ this.positionInSegment = position;
+ }
+
+ protected void clear() {
+ this.currentSegment = null;
+ this.positionInSegment = 0;
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Data Output Specific methods
+ // --------------------------------------------------------------------------------------------
+
+ @Override
+ public void write(int b) throws IOException {
+ writeByte(b);
+ }
+
+ @Override
+ public void write(byte[] b) throws IOException {
+ write(b, 0, b.length);
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ int remaining = this.segmentSize - this.positionInSegment;
+ if (remaining >= len) {
+ this.currentSegment.put(this.positionInSegment, b, off, len);
+ this.positionInSegment += len;
+ }
+ else {
+ if (remaining == 0) {
+ advance();
+ remaining = this.segmentSize - this.positionInSegment;
+ }
+ while (true) {
+ int toPut = Math.min(remaining, len);
+ this.currentSegment.put(this.positionInSegment, b, off, toPut);
+ off += toPut;
+ len -= toPut;
+
+ if (len > 0) {
+ this.positionInSegment = this.segmentSize;
+ advance();
+ remaining = this.segmentSize - this.positionInSegment;
+ }
+ else {
+ this.positionInSegment += toPut;
+ break;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void writeBoolean(boolean v) throws IOException {
+ writeByte(v ? 1 : 0);
+ }
+
+ @Override
+ public void writeByte(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize) {
+ this.currentSegment.put(this.positionInSegment++, (byte) v);
+ }
+ else {
+ advance();
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeShort(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 1) {
+ this.currentSegment.putShortBigEndian(this.positionInSegment, (short) v);
+ this.positionInSegment += 2;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeShort(v);
+ }
+ else {
+ writeByte(v >> 8);
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeChar(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 1) {
+ this.currentSegment.putCharBigEndian(this.positionInSegment, (char) v);
+ this.positionInSegment += 2;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeChar(v);
+ }
+ else {
+ writeByte(v >> 8);
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeInt(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 3) {
+ this.currentSegment.putIntBigEndian(this.positionInSegment, v);
+ this.positionInSegment += 4;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeInt(v);
+ }
+ else {
+ writeByte(v >> 24);
+ writeByte(v >> 16);
+ writeByte(v >> 8);
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeLong(long v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 7) {
+ this.currentSegment.putLongBigEndian(this.positionInSegment, v);
+ this.positionInSegment += 8;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeLong(v);
+ }
+ else {
+ writeByte((int) (v >> 56));
+ writeByte((int) (v >> 48));
+ writeByte((int) (v >> 40));
+ writeByte((int) (v >> 32));
+ writeByte((int) (v >> 24));
+ writeByte((int) (v >> 16));
+ writeByte((int) (v >> 8));
+ writeByte((int) v);
+ }
+ }
+
+ @Override
+ public void writeFloat(float v) throws IOException {
+ writeInt(Float.floatToRawIntBits(v));
+ }
+
+ @Override
+ public void writeDouble(double v) throws IOException {
+ writeLong(Double.doubleToRawLongBits(v));
+ }
+
+ @Override
+ public void writeBytes(String s) throws IOException {
+ for (int i = 0; i < s.length(); i++) {
+ writeByte(s.charAt(i));
+ }
+ }
+
+ @Override
+ public void writeChars(String s) throws IOException {
+ for (int i = 0; i < s.length(); i++) {
+ writeChar(s.charAt(i));
+ }
+ }
+
+ @Override
+ public void writeUTF(String str) throws IOException {
+ int strlen = str.length();
+ int utflen = 0;
+ int c, count = 0;
+
+ /* use charAt instead of copying String to char array */
+ for (int i = 0; i < strlen; i++) {
+ c = str.charAt(i);
+ if ((c >= 0x0001) && (c <= 0x007F)) {
+ utflen++;
+ } else if (c > 0x07FF) {
+ utflen += 3;
+ } else {
+ utflen += 2;
+ }
+ }
+
+ if (utflen > 65535) {
+ throw new UTFDataFormatException("encoded string too long: " + utflen + " memory");
+ }
+
+ if (this.utfBuffer == null || this.utfBuffer.length < utflen + 2) {
+ this.utfBuffer = new byte[utflen + 2];
+ }
+ final byte[] bytearr = this.utfBuffer;
+
+ bytearr[count++] = (byte) ((utflen >>> 8) & 0xFF);
+ bytearr[count++] = (byte) (utflen & 0xFF);
+
+ int i = 0;
+ for (i = 0; i < strlen; i++) {
+ c = str.charAt(i);
+ if (!((c >= 0x0001) && (c <= 0x007F))) {
+ break;
+ }
+ bytearr[count++] = (byte) c;
+ }
+
+ for (; i < strlen; i++) {
+ c = str.charAt(i);
+ if ((c >= 0x0001) && (c <= 0x007F)) {
+ bytearr[count++] = (byte) c;
+
+ } else if (c > 0x07FF) {
+ bytearr[count++] = (byte) (0xE0 | ((c >> 12) & 0x0F));
+ bytearr[count++] = (byte) (0x80 | ((c >> 6) & 0x3F));
+ bytearr[count++] = (byte) (0x80 | (c & 0x3F));
+ } else {
+ bytearr[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F));
+ bytearr[count++] = (byte) (0x80 | (c & 0x3F));
+ }
+ }
+
+ write(bytearr, 0, utflen + 2);
+ }
+
+ @Override
+ public void skipBytesToWrite(int numBytes) throws IOException {
+ while (numBytes > 0) {
+ final int remaining = this.segmentSize - this.positionInSegment;
+ if (numBytes <= remaining) {
+ this.positionInSegment += numBytes;
+ return;
+ }
+ this.positionInSegment = this.segmentSize;
+ advance();
+ numBytes -= remaining;
+ }
+ }
+
+ @Override
+ public void write(DataInputView source, int numBytes) throws IOException {
+ while (numBytes > 0) {
+ final int remaining = this.segmentSize - this.positionInSegment;
+ if (numBytes <= remaining) {
+ this.currentSegment.put(source, this.positionInSegment, numBytes);
+ this.positionInSegment += numBytes;
+ return;
+ }
+
+ if (remaining > 0) {
+ this.currentSegment.put(source, this.positionInSegment, remaining);
+ this.positionInSegment = this.segmentSize;
+ numBytes -= remaining;
+ }
+
+ advance();
+ }
+ }
+}
[13/15] flink git commit: [FLINK-1320] [core] Add an off-heap variant
of the managed memory
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java b/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java
new file mode 100644
index 0000000..4c92234
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory;
+
+import java.nio.ByteBuffer;
+import java.util.Random;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class EndiannessAccessChecks {
+
+ @Test
+ public void testHeapSegment() {
+ try {
+ testBigAndLittleEndianAccessUnaligned(new HeapMemorySegment(new byte[11111]));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testHybridOnHeapSegment() {
+ try {
+ testBigAndLittleEndianAccessUnaligned(new HybridMemorySegment(new byte[11111]));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testHybridOffHeapSegment() {
+ try {
+ testBigAndLittleEndianAccessUnaligned(new HybridMemorySegment(ByteBuffer.allocateDirect(11111)));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ private void testBigAndLittleEndianAccessUnaligned(MemorySegment segment) {
+ final Random rnd = new Random();
+
+ // longs
+ {
+ final long seed = rnd.nextLong();
+
+ rnd.setSeed(seed);
+ for (int i = 0; i < 10000; i++) {
+ long val = rnd.nextLong();
+ int pos = rnd.nextInt(segment.size - 7);
+
+ segment.putLongLittleEndian(pos, val);
+ long r = segment.getLongBigEndian(pos);
+ assertEquals(val, Long.reverseBytes(r));
+
+ segment.putLongBigEndian(pos, val);
+ r = segment.getLongLittleEndian(pos);
+ assertEquals(val, Long.reverseBytes(r));
+ }
+ }
+
+ // ints
+ {
+ final long seed = rnd.nextLong();
+
+ rnd.setSeed(seed);
+ for (int i = 0; i < 10000; i++) {
+ int val = rnd.nextInt();
+ int pos = rnd.nextInt(segment.size - 3);
+
+ segment.putIntLittleEndian(pos, val);
+ int r = segment.getIntBigEndian(pos);
+ assertEquals(val, Integer.reverseBytes(r));
+
+ segment.putIntBigEndian(pos, val);
+ r = segment.getIntLittleEndian(pos);
+ assertEquals(val, Integer.reverseBytes(r));
+ }
+ }
+
+ // shorts
+ {
+ final long seed = rnd.nextLong();
+
+ rnd.setSeed(seed);
+ for (int i = 0; i < 10000; i++) {
+ short val = (short) rnd.nextInt();
+ int pos = rnd.nextInt(segment.size - 1);
+
+ segment.putShortLittleEndian(pos, val);
+ short r = segment.getShortBigEndian(pos);
+ assertEquals(val, Short.reverseBytes(r));
+
+ segment.putShortBigEndian(pos, val);
+ r = segment.getShortLittleEndian(pos);
+ assertEquals(val, Short.reverseBytes(r));
+ }
+ }
+
+ // chars
+ {
+ final long seed = rnd.nextLong();
+
+ rnd.setSeed(seed);
+ for (int i = 0; i < 10000; i++) {
+ char val = (char) rnd.nextInt();
+ int pos = rnd.nextInt(segment.size - 1);
+
+ segment.putCharLittleEndian(pos, val);
+ char r = segment.getCharBigEndian(pos);
+ assertEquals(val, Character.reverseBytes(r));
+
+ segment.putCharBigEndian(pos, val);
+ r = segment.getCharLittleEndian(pos);
+ assertEquals(val, Character.reverseBytes(r));
+ }
+ }
+
+ // floats
+ {
+ final long seed = rnd.nextLong();
+
+ rnd.setSeed(seed);
+ for (int i = 0; i < 10000; i++) {
+ float val = rnd.nextFloat();
+ int pos = rnd.nextInt(segment.size - 3);
+
+ segment.putFloatLittleEndian(pos, val);
+ float r = segment.getFloatBigEndian(pos);
+ float reversed = Float.intBitsToFloat(Integer.reverseBytes(Float.floatToRawIntBits(r)));
+ assertEquals(val, reversed, 0.0f);
+
+ segment.putFloatBigEndian(pos, val);
+ r = segment.getFloatLittleEndian(pos);
+ reversed = Float.intBitsToFloat(Integer.reverseBytes(Float.floatToRawIntBits(r)));
+ assertEquals(val, reversed, 0.0f);
+ }
+ }
+
+ // doubles
+ {
+ final long seed = rnd.nextLong();
+
+ rnd.setSeed(seed);
+ for (int i = 0; i < 10000; i++) {
+ double val = rnd.nextDouble();
+ int pos = rnd.nextInt(segment.size - 7);
+
+ segment.putDoubleLittleEndian(pos, val);
+ double r = segment.getDoubleBigEndian(pos);
+ double reversed = Double.longBitsToDouble(Long.reverseBytes(Double.doubleToRawLongBits(r)));
+ assertEquals(val, reversed, 0.0f);
+
+ segment.putDoubleBigEndian(pos, val);
+ r = segment.getDoubleLittleEndian(pos);
+ reversed = Double.longBitsToDouble(Long.reverseBytes(Double.doubleToRawLongBits(r)));
+ assertEquals(val, reversed, 0.0f);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/HeapMemorySegmentTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/HeapMemorySegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/HeapMemorySegmentTest.java
new file mode 100644
index 0000000..d7a5b03
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/HeapMemorySegmentTest.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.*;
+
+@RunWith(Parameterized.class)
+public class HeapMemorySegmentTest extends MemorySegmentTestBase {
+
+ public HeapMemorySegmentTest(int pageSize) {
+ super(pageSize);
+ }
+
+ @Override
+ MemorySegment createSegment(int size) {
+ return new HeapMemorySegment(new byte[size]);
+ }
+
+ @Override
+ MemorySegment createSegment(int size, Object owner) {
+ return new HeapMemorySegment(new byte[size], owner);
+ }
+
+ @Test
+ public void testHeapSegmentSpecifics() {
+ try {
+ final byte[] buffer = new byte[411];
+ HeapMemorySegment seg = new HeapMemorySegment(buffer);
+
+ assertFalse(seg.isFreed());
+ assertFalse(seg.isOffHeap());
+ assertEquals(buffer.length, seg.size());
+ assertTrue(buffer == seg.getArray());
+
+ ByteBuffer buf1 = seg.wrap(1, 2);
+ ByteBuffer buf2 = seg.wrap(3, 4);
+
+ assertTrue(buf1 != buf2);
+ assertEquals(1, buf1.position());
+ assertEquals(3, buf1.limit());
+ assertEquals(3, buf2.position());
+ assertEquals(7, buf2.limit());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java
new file mode 100644
index 0000000..b09697e
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@RunWith(Parameterized.class)
+public class HybridOffHeapMemorySegmentTest extends MemorySegmentTestBase {
+
+ public HybridOffHeapMemorySegmentTest(int pageSize) {
+ super(pageSize);
+ }
+
+ @Override
+ MemorySegment createSegment(int size) {
+ return new HybridMemorySegment(ByteBuffer.allocateDirect(size));
+ }
+
+ @Override
+ MemorySegment createSegment(int size, Object owner) {
+ return new HybridMemorySegment(ByteBuffer.allocateDirect(size), owner);
+ }
+
+ @Test
+ public void testHybridHeapSegmentSpecifics() {
+ try {
+ final ByteBuffer buffer = ByteBuffer.allocateDirect(411);
+ HybridMemorySegment seg = new HybridMemorySegment(buffer);
+
+ assertFalse(seg.isFreed());
+ assertTrue(seg.isOffHeap());
+ assertEquals(buffer.capacity(), seg.size());
+ assertTrue(buffer == seg.getOffHeapBuffer());
+
+ try {
+ seg.getArray();
+ fail("should throw an exception");
+ }
+ catch (IllegalStateException e) {
+ // expected
+ }
+
+ ByteBuffer buf1 = seg.wrap(1, 2);
+ ByteBuffer buf2 = seg.wrap(3, 4);
+
+ assertTrue(buf1 != buffer);
+ assertTrue(buf2 != buffer);
+ assertTrue(buf1 != buf2);
+ assertEquals(1, buf1.position());
+ assertEquals(3, buf1.limit());
+ assertEquals(3, buf2.position());
+ assertEquals(7, buf2.limit());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java
new file mode 100644
index 0000000..55d333e
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@RunWith(Parameterized.class)
+public class HybridOnHeapMemorySegmentTest extends MemorySegmentTestBase {
+
+ public HybridOnHeapMemorySegmentTest(int pageSize) {
+ super(pageSize);
+ }
+
+ @Override
+ MemorySegment createSegment(int size) {
+ return new HybridMemorySegment(new byte[size]);
+ }
+
+ @Override
+ MemorySegment createSegment(int size, Object owner) {
+ return new HybridMemorySegment(new byte[size], owner);
+ }
+
+ @Test
+ public void testHybridHeapSegmentSpecifics() {
+ try {
+ final byte[] buffer = new byte[411];
+ HybridMemorySegment seg = new HybridMemorySegment(buffer);
+
+ assertFalse(seg.isFreed());
+ assertFalse(seg.isOffHeap());
+ assertEquals(buffer.length, seg.size());
+ assertTrue(buffer == seg.getArray());
+
+ try {
+ seg.getOffHeapBuffer();
+ fail("should throw an exception");
+ }
+ catch (IllegalStateException e) {
+ // expected
+ }
+
+ ByteBuffer buf1 = seg.wrap(1, 2);
+ ByteBuffer buf2 = seg.wrap(3, 4);
+
+ assertTrue(buf1 != buf2);
+ assertEquals(1, buf1.position());
+ assertEquals(3, buf1.limit());
+ assertEquals(3, buf2.position());
+ assertEquals(7, buf2.limit());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java
new file mode 100644
index 0000000..f50322c
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory;
+
+import org.junit.Test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.nio.ByteBuffer;
+
+/**
+ * Tests for the sanity checks of the memory segments.
+ */
+public class MemorySegmentChecksTest {
+
+ @Test(expected = NullPointerException.class)
+ public void testHeapNullBuffer1() {
+ new HeapMemorySegment(null);
+ }
+
+ @Test(expected = NullPointerException.class)
+ public void testHeapNullBuffer2() {
+ new HeapMemorySegment(null, new Object());
+ }
+
+ @Test(expected = NullPointerException.class)
+ public void testHybridHeapNullBuffer1() {
+ new HybridMemorySegment((byte[]) null);
+ }
+
+ @Test(expected = NullPointerException.class)
+ public void testHybridHeapNullBuffer2() {
+ new HybridMemorySegment((byte[]) null, new Object());
+ }
+
+ @Test(expected = NullPointerException.class)
+ public void testHybridOffHeapNullBuffer1() {
+ new HybridMemorySegment((ByteBuffer) null);
+ }
+
+ @Test(expected = NullPointerException.class)
+ public void testHybridOffHeapNullBuffer2() {
+ new HybridMemorySegment((ByteBuffer) null, new Object());
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testHybridNonDirectBuffer() {
+ new HybridMemorySegment(ByteBuffer.allocate(1024));
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testZeroAddress(){
+ new MockSegment(0L, 4*1024, null);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testNegativeAddress(){
+ new MockSegment(-1L, 4*1024, null);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testTooLargeAddress(){
+ new MockSegment(Long.MAX_VALUE - 8*1024, 4*1024, null);
+ }
+
+ // ------------------------------------------------------------------------
+
+ final class MockSegment extends MemorySegment {
+
+ MockSegment(long offHeapAddress, int size, Object owner) {
+ super(offHeapAddress, size, owner);
+ }
+
+ @Override
+ public ByteBuffer wrap(int offset, int length) {
+ return null;
+ }
+
+ @Override
+ public byte get(int index) {
+ return 0;
+ }
+
+ @Override
+ public void put(int index, byte b) {}
+
+ @Override
+ public void get(int index, byte[] dst) {}
+
+ @Override
+ public void put(int index, byte[] src) {}
+
+ @Override
+ public void get(int index, byte[] dst, int offset, int length) {}
+
+ @Override
+ public void put(int index, byte[] src, int offset, int length) {}
+
+ @Override
+ public boolean getBoolean(int index) {
+ return false;
+ }
+
+ @Override
+ public void putBoolean(int index, boolean value) {}
+
+ @Override
+ public void get(DataOutput out, int offset, int length) {}
+
+ @Override
+ public void put(DataInput in, int offset, int length) {}
+
+ @Override
+ public void get(int offset, ByteBuffer target, int numBytes) {}
+
+ @Override
+ public void put(int offset, ByteBuffer source, int numBytes) {}
+ };
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java
new file mode 100644
index 0000000..10c3622
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java
@@ -0,0 +1,2571 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.BufferOverflowException;
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Random;
+
+import static org.junit.Assert.*;
+
+/**
+ * Tests for the access and transfer methods of the HeapMemorySegment.
+ */
+public abstract class MemorySegmentTestBase {
+
+ private final Random random = new Random();
+
+ private final int pageSize;
+
+
+ public MemorySegmentTestBase(int pageSize) {
+ this.pageSize = pageSize;
+ }
+
+ // ------------------------------------------------------------------------
+ // Access to primitives
+ // ------------------------------------------------------------------------
+
+ abstract MemorySegment createSegment(int size);
+
+ abstract MemorySegment createSegment(int size, Object owner);
+
+ // ------------------------------------------------------------------------
+ // Access to primitives
+ // ------------------------------------------------------------------------
+
+ @Test
+ public void testByteAccess() {
+ try {
+ final MemorySegment segment = createSegment(pageSize);
+
+ // test exceptions
+ try {
+ segment.put(-1, (byte) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(pageSize, (byte) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(Integer.MAX_VALUE, (byte) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(Integer.MIN_VALUE, (byte) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(pageSize);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // test expected correct behavior, sequential access
+
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i < pageSize; i++) {
+ segment.put(i, (byte) random.nextInt());
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i < pageSize; i++) {
+ assertEquals((byte) random.nextInt(), segment.get(i));
+ }
+
+ // test expected correct behavior, random access
+
+ random.setSeed(seed);
+ boolean[] occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize);
+
+ if (occupied[pos]) {
+ continue;
+ } else {
+ occupied[pos] = true;
+ }
+
+ segment.put(pos, (byte) random.nextInt());
+ }
+
+ random.setSeed(seed);
+ occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize);
+
+ if (occupied[pos]) {
+ continue;
+ } else {
+ occupied[pos] = true;
+ }
+
+ assertEquals((byte) random.nextInt(), segment.get(pos));
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+
+ }
+
+ @Test
+ public void testBooleanAccess() {
+ try {
+ final MemorySegment segment = createSegment(pageSize);
+
+ // test exceptions
+ try {
+ segment.putBoolean(-1, false);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putBoolean(pageSize, false);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putBoolean(Integer.MAX_VALUE, false);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putBoolean(Integer.MIN_VALUE, false);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getBoolean(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getBoolean(pageSize);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getBoolean(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getBoolean(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // test expected correct behavior, sequential access
+
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i < pageSize; i++) {
+ segment.putBoolean(i, random.nextBoolean());
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i < pageSize; i++) {
+ assertEquals(random.nextBoolean(), segment.getBoolean(i));
+ }
+
+ // test expected correct behavior, random access
+
+ random.setSeed(seed);
+ boolean[] occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize);
+
+ if (occupied[pos]) {
+ continue;
+ } else {
+ occupied[pos] = true;
+ }
+
+ segment.putBoolean(pos, random.nextBoolean());
+ }
+
+ random.setSeed(seed);
+ occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize);
+
+ if (occupied[pos]) {
+ continue;
+ } else {
+ occupied[pos] = true;
+ }
+
+ assertEquals(random.nextBoolean(), segment.getBoolean(pos));
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testCharAccess() {
+ try {
+ final MemorySegment segment = createSegment(pageSize);
+
+ // test exceptions
+
+ try {
+ segment.putChar(-1, 'a');
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putChar(pageSize, 'a');
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putChar(Integer.MIN_VALUE, 'a');
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putChar(Integer.MAX_VALUE, 'a');
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putChar(Integer.MAX_VALUE - 1, 'a');
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getChar(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getChar(pageSize);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getChar(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getChar(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getChar(Integer.MAX_VALUE - 1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // test expected correct behavior, sequential access
+
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i <= pageSize - 2; i += 2) {
+ segment.putChar(i, (char) (random.nextInt(Character.MAX_VALUE)));
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i <= pageSize - 2; i += 2) {
+ assertEquals((char) (random.nextInt(Character.MAX_VALUE)), segment.getChar(i));
+ }
+
+ // test expected correct behavior, random access
+
+ random.setSeed(seed);
+ boolean[] occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize - 1);
+
+ if (occupied[pos] || occupied[pos + 1]) {
+ continue;
+ } else {
+ occupied[pos] = true;
+ occupied[pos+1] = true;
+ }
+
+ segment.putChar(pos, (char) (random.nextInt(Character.MAX_VALUE)));
+ }
+
+ random.setSeed(seed);
+ occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize - 1);
+
+ if (occupied[pos] || occupied[pos + 1]) {
+ continue;
+ } else {
+ occupied[pos] = true;
+ occupied[pos+1] = true;
+ }
+
+ assertEquals((char) (random.nextInt(Character.MAX_VALUE)), segment.getChar(pos));
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testShortAccess() {
+ try {
+ final MemorySegment segment = createSegment(pageSize);
+
+ // test exceptions
+
+ try {
+ segment.putShort(-1, (short) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putShort(pageSize, (short) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putShort(Integer.MIN_VALUE, (short) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putShort(Integer.MAX_VALUE, (short) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putShort(Integer.MAX_VALUE - 1, (short) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getShort(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getShort(pageSize);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getShort(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getShort(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getShort(Integer.MAX_VALUE - 1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // test expected correct behavior, sequential access
+
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i <= pageSize - 2; i += 2) {
+ segment.putShort(i, (short) random.nextInt());
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i <= pageSize - 2; i += 2) {
+ assertEquals((short) random.nextInt(), segment.getShort(i));
+ }
+
+ // test expected correct behavior, random access
+
+ random.setSeed(seed);
+ boolean[] occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize - 1);
+
+ if (occupied[pos] || occupied[pos + 1]) {
+ continue;
+ } else {
+ occupied[pos] = true;
+ occupied[pos+1] = true;
+ }
+
+ segment.putShort(pos, (short) random.nextInt());
+ }
+
+ random.setSeed(seed);
+ occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize - 1);
+
+ if (occupied[pos] || occupied[pos + 1]) {
+ continue;
+ } else {
+ occupied[pos] = true;
+ occupied[pos+1] = true;
+ }
+
+ assertEquals((short) random.nextInt(), segment.getShort(pos));
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testIntAccess() {
+ try {
+ final MemorySegment segment = createSegment(pageSize);
+
+ // test exceptions
+
+ try {
+ segment.putInt(-1, 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putInt(pageSize, 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putInt(pageSize - 3, 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putInt(Integer.MIN_VALUE, 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putInt(Integer.MAX_VALUE, 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putInt(Integer.MAX_VALUE - 3, 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getInt(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getInt(pageSize);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getInt(pageSize - 3);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getInt(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getInt(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getInt(Integer.MAX_VALUE - 3);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // test expected correct behavior, sequential access
+
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i <= pageSize - 4; i += 4) {
+ segment.putInt(i, random.nextInt());
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i <= pageSize - 4; i += 4) {
+ assertEquals(random.nextInt(), segment.getInt(i));
+ }
+
+ // test expected correct behavior, random access
+
+ random.setSeed(seed);
+ boolean[] occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize - 3);
+
+ if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3]) {
+ continue;
+ } else {
+ occupied[pos] = true;
+ occupied[pos+1] = true;
+ occupied[pos+2] = true;
+ occupied[pos+3] = true;
+ }
+
+ segment.putInt(pos, random.nextInt());
+ }
+
+ random.setSeed(seed);
+ occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize - 3);
+
+ if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3]) {
+ continue;
+ } else {
+ occupied[pos] = true;
+ occupied[pos+1] = true;
+ occupied[pos+2] = true;
+ occupied[pos+3] = true;
+ }
+
+ assertEquals(random.nextInt(), segment.getInt(pos));
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testLongAccess() {
+ try {
+ final MemorySegment segment = createSegment(pageSize);
+
+ // test exceptions
+
+ try {
+ segment.putLong(-1, 0L);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putLong(pageSize, 0L);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putLong(pageSize - 7, 0L);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putLong(Integer.MIN_VALUE, 0L);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putLong(Integer.MAX_VALUE, 0L);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putLong(Integer.MAX_VALUE - 7, 0L);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getLong(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getLong(pageSize);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getLong(pageSize - 7);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getLong(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getLong(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getLong(Integer.MAX_VALUE - 7);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // test expected correct behavior, sequential access
+
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i <= pageSize - 8; i += 8) {
+ segment.putLong(i, random.nextLong());
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i <= pageSize - 8; i += 8) {
+ assertEquals(random.nextLong(), segment.getLong(i));
+ }
+
+ // test expected correct behavior, random access
+
+ random.setSeed(seed);
+ boolean[] occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize - 7);
+
+ if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3] ||
+ occupied[pos + 4] || occupied[pos + 5] || occupied[pos + 6] || occupied[pos + 7])
+ {
+ continue;
+ }
+ else {
+ occupied[pos] = true;
+ occupied[pos+1] = true;
+ occupied[pos+2] = true;
+ occupied[pos+3] = true;
+ occupied[pos+4] = true;
+ occupied[pos+5] = true;
+ occupied[pos+6] = true;
+ occupied[pos+7] = true;
+ }
+
+ segment.putLong(pos, random.nextLong());
+ }
+
+ random.setSeed(seed);
+ occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize - 7);
+
+ if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3] ||
+ occupied[pos + 4] || occupied[pos + 5] || occupied[pos + 6] || occupied[pos + 7])
+ {
+ continue;
+ }
+ else {
+ occupied[pos] = true;
+ occupied[pos+1] = true;
+ occupied[pos+2] = true;
+ occupied[pos+3] = true;
+ occupied[pos+4] = true;
+ occupied[pos+5] = true;
+ occupied[pos+6] = true;
+ occupied[pos+7] = true;
+ }
+
+ assertEquals(random.nextLong(), segment.getLong(pos));
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testFloatAccess() {
+ try {
+ final MemorySegment segment = createSegment(pageSize);
+
+ // test exceptions
+
+ try {
+ segment.putFloat(-1, 0.0f);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putFloat(pageSize, 0.0f);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putFloat(pageSize - 3, 0.0f);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putFloat(Integer.MIN_VALUE, 0.0f);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putFloat(Integer.MAX_VALUE, 0.0f);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putFloat(Integer.MAX_VALUE - 3, 0.0f);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getFloat(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getFloat(pageSize);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getFloat(pageSize - 3);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getFloat(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getFloat(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getFloat(Integer.MAX_VALUE - 3);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // test expected correct behavior, sequential access
+
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i <= pageSize - 4; i += 4) {
+ segment.putFloat(i, random.nextFloat());
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i <= pageSize - 4; i += 4) {
+ assertEquals(random.nextFloat(), segment.getFloat(i), 0.0);
+ }
+
+ // test expected correct behavior, random access
+
+ random.setSeed(seed);
+ boolean[] occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize - 3);
+
+ if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3]) {
+ continue;
+ } else {
+ occupied[pos] = true;
+ occupied[pos+1] = true;
+ occupied[pos+2] = true;
+ occupied[pos+3] = true;
+ }
+
+ segment.putFloat(pos, random.nextFloat());
+ }
+
+ random.setSeed(seed);
+ occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize - 3);
+
+ if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3]) {
+ continue;
+ } else {
+ occupied[pos] = true;
+ occupied[pos+1] = true;
+ occupied[pos+2] = true;
+ occupied[pos+3] = true;
+ }
+
+ assertEquals(random.nextFloat(), segment.getFloat(pos), 0.0);
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testDoubleAccess() {
+ try {
+ final MemorySegment segment = createSegment(pageSize);
+
+ // test exceptions
+ try {
+ segment.putDouble(-1, 0.0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putDouble(pageSize, 0.0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putDouble(pageSize - 7, 0.0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putDouble(Integer.MIN_VALUE, 0.0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putDouble(Integer.MAX_VALUE, 0.0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putDouble(Integer.MAX_VALUE - 7, 0.0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getDouble(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getDouble(pageSize);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getDouble(pageSize - 7);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getDouble(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getDouble(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getDouble(Integer.MAX_VALUE - 7);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // test expected correct behavior, sequential access
+
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ for (int i = 0; i <= pageSize - 8; i += 8) {
+ segment.putDouble(i, random.nextDouble());
+ }
+
+ random.setSeed(seed);
+ for (int i = 0; i <= pageSize - 8; i += 8) {
+ assertEquals(random.nextDouble(), segment.getDouble(i), 0.0);
+ }
+
+ // test expected correct behavior, random access
+
+ random.setSeed(seed);
+ boolean[] occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize - 7);
+
+ if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3] ||
+ occupied[pos + 4] || occupied[pos + 5] || occupied[pos + 6] || occupied[pos + 7])
+ {
+ continue;
+ }
+ else {
+ occupied[pos] = true;
+ occupied[pos+1] = true;
+ occupied[pos+2] = true;
+ occupied[pos+3] = true;
+ occupied[pos+4] = true;
+ occupied[pos+5] = true;
+ occupied[pos+6] = true;
+ occupied[pos+7] = true;
+ }
+
+ segment.putDouble(pos, random.nextDouble());
+ }
+
+ random.setSeed(seed);
+ occupied = new boolean[pageSize];
+
+ for (int i = 0; i < 1000; i++) {
+ int pos = random.nextInt(pageSize - 7);
+
+ if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3] ||
+ occupied[pos + 4] || occupied[pos + 5] || occupied[pos + 6] || occupied[pos + 7])
+ {
+ continue;
+ }
+ else {
+ occupied[pos] = true;
+ occupied[pos+1] = true;
+ occupied[pos+2] = true;
+ occupied[pos+3] = true;
+ occupied[pos+4] = true;
+ occupied[pos+5] = true;
+ occupied[pos+6] = true;
+ occupied[pos+7] = true;
+ }
+
+ assertEquals(random.nextDouble(), segment.getDouble(pos), 0.0);
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Bulk Byte Movements
+ // ------------------------------------------------------------------------
+
+ @Test
+ public void testBulkBytePutExceptions() {
+ try {
+ final MemorySegment segment = createSegment(pageSize);
+
+ byte[] bytes = new byte[pageSize / 4 + (pageSize%4)];
+ random.nextBytes(bytes);
+
+ // wrong positions into memory segment
+
+ try {
+ segment.put(-1, bytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(-1, bytes, 4, 5);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(Integer.MIN_VALUE, bytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(Integer.MIN_VALUE, bytes, 4, 5);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(pageSize, bytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(pageSize, bytes, 6, 44);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(pageSize - bytes.length + 1, bytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(pageSize - 5, bytes, 3, 6);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(Integer.MAX_VALUE, bytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(Integer.MAX_VALUE, bytes, 10, 20);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(Integer.MAX_VALUE - bytes.length + 1, bytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(Integer.MAX_VALUE - 11, bytes, 11, 11);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(3 * (pageSize / 4) + 1, bytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(3 * (pageSize / 4) + 2, bytes, 0, bytes.length - 1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(7 * (pageSize / 8) + 1, bytes, 0, bytes.length / 2);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // wrong source array positions / lengths
+
+ try {
+ segment.put(0, bytes, -1, 1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(0, bytes, -1, bytes.length + 1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(0, bytes, Integer.MIN_VALUE, bytes.length);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(0, bytes, Integer.MAX_VALUE, bytes.length);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(0, bytes, Integer.MAX_VALUE - bytes.length + 1, bytes.length);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // case where negative offset and negative index compensate each other
+ try {
+ segment.put(-2, bytes, -1, bytes.length / 2);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testBulkByteGetExceptions() {
+ try {
+ final MemorySegment segment = createSegment(pageSize);
+
+ byte[] bytes = new byte[pageSize / 4];
+
+ // wrong positions into memory segment
+
+ try {
+ segment.get(-1, bytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(-1, bytes, 4, 5);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(Integer.MIN_VALUE, bytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(Integer.MIN_VALUE, bytes, 4, 5);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(pageSize, bytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(pageSize, bytes, 6, 44);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(pageSize - bytes.length + 1, bytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(pageSize - 5, bytes, 3, 6);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(Integer.MAX_VALUE, bytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(Integer.MAX_VALUE, bytes, 10, 20);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(Integer.MAX_VALUE - bytes.length + 1, bytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(Integer.MAX_VALUE - 11, bytes, 11, 11);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(3 * (pageSize / 4) + 1, bytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(3 * (pageSize / 4) + 2, bytes, 0, bytes.length - 1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(7 * (pageSize / 8) + 1, bytes, 0, bytes.length / 2);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // wrong source array positions / lengths
+
+ try {
+ segment.get(0, bytes, -1, 1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(0, bytes, -1, bytes.length + 1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(0, bytes, Integer.MIN_VALUE, bytes.length);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(0, bytes, Integer.MAX_VALUE, bytes.length);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(0, bytes, Integer.MAX_VALUE - bytes.length + 1, bytes.length);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // case where negative offset and negative index compensate each other
+ try {
+ segment.get(-2, bytes, -1, bytes.length / 2);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testBulkByteAccess() {
+ try {
+ // test expected correct behavior with default offset / length
+ {
+ final MemorySegment segment = createSegment(pageSize);
+ long seed = random.nextLong();
+
+ random.setSeed(seed);
+ byte[] src = new byte[pageSize / 8];
+ for (int i = 0; i < 8; i++) {
+ random.nextBytes(src);
+ segment.put(i * (pageSize / 8), src);
+ }
+
+ random.setSeed(seed);
+ byte[] expected = new byte[pageSize / 8];
+ byte[] actual = new byte[pageSize / 8];
+ for (int i = 0; i < 8; i++) {
+ random.nextBytes(expected);
+ segment.get(i * (pageSize / 8), actual);
+
+ assertArrayEquals(expected, actual);
+ }
+ }
+
+ // test expected correct behavior with specific offset / length
+ {
+ final MemorySegment segment = createSegment(pageSize);
+ byte[] expected = new byte[pageSize];
+ random.nextBytes(expected);
+
+ for (int i = 0; i < 16; i++) {
+ segment.put(i * (pageSize / 16), expected, i * (pageSize / 16),
+ pageSize / 16);
+ }
+
+ byte[] actual = new byte[pageSize];
+ for (int i = 0; i < 16; i++) {
+ segment.get(i * (pageSize / 16), actual, i * (pageSize / 16),
+ pageSize / 16);
+ }
+
+ assertArrayEquals(expected, actual);
+ }
+
+ // put segments of various lengths to various positions
+ {
+ final MemorySegment segment = createSegment(pageSize);
+ byte[] expected = new byte[pageSize];
+
+ for (int i = 0; i < 200; i++) {
+ int numBytes = random.nextInt(pageSize - 10) + 1;
+ int pos = random.nextInt(pageSize - numBytes + 1);
+
+ byte[] data = new byte[(random.nextInt(3) + 1) * numBytes];
+ int dataStartPos = random.nextInt(data.length - numBytes + 1);
+
+ random.nextBytes(data);
+
+ // copy to the expected
+ System.arraycopy(data, dataStartPos, expected, pos, numBytes);
+
+ // put to the memory segment
+ segment.put(pos, data, dataStartPos, numBytes);
+ }
+
+ byte[] validation = new byte[pageSize];
+ segment.get(0, validation);
+
+ assertArrayEquals(expected, validation);
+ }
+
+ // get segments with various contents
+ {
+ final MemorySegment segment = createSegment(pageSize);
+ byte[] contents = new byte[pageSize];
+ random.nextBytes(contents);
+ segment.put(0, contents);
+
+ for (int i = 0; i < 200; i++) {
+ int numBytes = random.nextInt(pageSize / 8) + 1;
+ int pos = random.nextInt(pageSize - numBytes + 1);
+
+ byte[] data = new byte[(random.nextInt(3) + 1) * numBytes];
+ int dataStartPos = random.nextInt(data.length - numBytes + 1);
+
+ segment.get(pos, data, dataStartPos, numBytes);
+
+ byte[] expected = Arrays.copyOfRange(contents, pos, pos + numBytes);
+ byte[] validation = Arrays.copyOfRange(data, dataStartPos, dataStartPos + numBytes);
+ assertArrayEquals(expected, validation);
+ }
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Writing / Reading to/from DataInput / DataOutput
+ // ------------------------------------------------------------------------
+
+ @Test
+ public void testDataInputOutput() {
+ try {
+ MemorySegment seg = createSegment(pageSize);
+ byte[] contents = new byte[pageSize];
+ random.nextBytes(contents);
+ seg.put(0, contents);
+
+ ByteArrayOutputStream buffer = new ByteArrayOutputStream(pageSize);
+ DataOutputStream out = new DataOutputStream(buffer);
+
+ // write the segment in chunks into the stream
+ int pos = 0;
+ while (pos < pageSize) {
+ int len = random.nextInt(200);
+ len = Math.min(len, pageSize - pos);
+ seg.get(out, pos, len);
+ pos += len;
+ }
+
+ // verify that we wrote the same bytes
+ byte[] result = buffer.toByteArray();
+ assertArrayEquals(contents, result);
+
+ // re-read the bytes into a new memory segment
+ MemorySegment reader = createSegment(pageSize);
+ DataInputStream in = new DataInputStream(new ByteArrayInputStream(result));
+
+ pos = 0;
+ while (pos < pageSize) {
+ int len = random.nextInt(200);
+ len = Math.min(len, pageSize - pos);
+ reader.put(in, pos, len);
+ pos += len;
+ }
+
+ byte[] targetBuffer = new byte[pageSize];
+ reader.get(0, targetBuffer);
+
+ assertArrayEquals(contents, targetBuffer);
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testDataInputOutputOutOfBounds() {
+ try {
+ final int segmentSize = 52;
+
+ // segment with random contents
+ MemorySegment seg = createSegment(segmentSize);
+ byte[] bytes = new byte[segmentSize];
+ random.nextBytes(bytes);
+ seg.put(0, bytes);
+
+ // out of bounds when writing
+ {
+ DataOutputStream out = new DataOutputStream(new ByteArrayOutputStream());
+
+ try {
+ seg.get(out, -1, segmentSize / 2);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ seg.get(out, segmentSize, segmentSize / 2);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ seg.get(out, -segmentSize, segmentSize / 2);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ seg.get(out, Integer.MIN_VALUE, segmentSize / 2);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ seg.get(out, Integer.MAX_VALUE, segmentSize / 2);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ }
+
+ // out of bounds when reading
+ {
+ DataInputStream in = new DataInputStream(new ByteArrayInputStream(new byte[segmentSize]));
+
+ try {
+ seg.put(in, -1, segmentSize / 2);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ seg.put(in, segmentSize, segmentSize / 2);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ seg.put(in, -segmentSize, segmentSize / 2);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ seg.put(in, Integer.MIN_VALUE, segmentSize / 2);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ seg.put(in, Integer.MAX_VALUE, segmentSize / 2);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testDataInputOutputStreamUnderflowOverflow() {
+ try {
+ final int segmentSize = 1337;
+
+ // segment with random contents
+ MemorySegment seg = createSegment(segmentSize);
+ byte[] bytes = new byte[segmentSize];
+ random.nextBytes(bytes);
+ seg.put(0, bytes);
+
+ // a stream that we cannot fully write to
+ DataOutputStream out = new DataOutputStream(new OutputStream() {
+
+ int bytesSoFar = 0;
+ @Override
+ public void write(int b) throws IOException {
+ bytesSoFar++;
+ if (bytesSoFar > segmentSize / 2) {
+ throw new IOException("overflow");
+ }
+ }
+ });
+
+ // write the segment in chunks into the stream
+ try {
+ int pos = 0;
+ while (pos < pageSize) {
+ int len = random.nextInt(segmentSize / 10);
+ len = Math.min(len, pageSize - pos);
+ seg.get(out, pos, len);
+ pos += len;
+ }
+ fail("Should fail with an IOException");
+ }
+ catch (IOException e) {
+ // expected
+ }
+
+ DataInputStream in = new DataInputStream(new ByteArrayInputStream(new byte[segmentSize / 2]));
+
+ try {
+ int pos = 0;
+ while (pos < pageSize) {
+ int len = random.nextInt(segmentSize / 10);
+ len = Math.min(len, pageSize - pos);
+ seg.put(in, pos, len);
+ pos += len;
+ }
+ fail("Should fail with an EOFException");
+ }
+ catch (EOFException e) {
+ // expected
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // ByteBuffer Ops
+ // ------------------------------------------------------------------------
+
+ @Test
+ public void testByteBufferGet() {
+ try {
+ testByteBufferGet(false);
+ testByteBufferGet(true);
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ private void testByteBufferGet(boolean directBuffer) {
+ MemorySegment seg = createSegment(pageSize);
+ byte[] bytes = new byte[pageSize];
+ random.nextBytes(bytes);
+ seg.put(0, bytes);
+
+ ByteBuffer target = directBuffer ?
+ ByteBuffer.allocateDirect(3 * pageSize) :
+ ByteBuffer.allocate(3 * pageSize);
+ target.position(2 * pageSize);
+
+ // transfer the segment in chunks into the byte buffer
+ int pos = 0;
+ while (pos < pageSize) {
+ int len = random.nextInt(pageSize / 10);
+ len = Math.min(len, pageSize - pos);
+ seg.get(pos, target, len);
+ pos += len;
+ }
+
+ // verify that we wrote the same bytes
+ byte[] result = new byte[pageSize];
+ target.position(2 * pageSize);
+ target.get(result);
+
+ assertArrayEquals(bytes, result);
+ }
+
+ @Test
+ public void testByteBufferPut() {
+ try {
+ testByteBufferPut(false);
+ testByteBufferPut(true);
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ private void testByteBufferPut(boolean directBuffer) {
+ byte[] bytes = new byte[pageSize];
+ random.nextBytes(bytes);
+
+ ByteBuffer source = directBuffer ?
+ ByteBuffer.allocateDirect(pageSize) :
+ ByteBuffer.allocate(pageSize);
+
+ source.put(bytes);
+ source.clear();
+
+ MemorySegment seg = createSegment(3 * pageSize);
+
+ int offset = 2 * pageSize;
+
+ // transfer the segment in chunks into the byte buffer
+ int pos = 0;
+ while (pos < pageSize) {
+ int len = random.nextInt(pageSize / 10);
+ len = Math.min(len, pageSize - pos);
+ seg.put(offset + pos, source, len);
+ pos += len;
+ }
+
+ // verify that we read the same bytes
+ byte[] result = new byte[pageSize];
+ seg.get(offset, result);
+
+ assertArrayEquals(bytes, result);
+ }
+
+ // ------------------------------------------------------------------------
+ // ByteBuffer Ops on sliced byte buffers
+ // ------------------------------------------------------------------------
+
+ @Test
+ public void testSlicedByteBufferGet() {
+ try {
+ testSlicedByteBufferGet(false);
+ testSlicedByteBufferGet(true);
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ private void testSlicedByteBufferGet(boolean directBuffer) {
+ MemorySegment seg = createSegment(pageSize);
+ byte[] bytes = new byte[pageSize];
+ random.nextBytes(bytes);
+ seg.put(0, bytes);
+
+ ByteBuffer target = directBuffer ?
+ ByteBuffer.allocateDirect(pageSize + 49) :
+ ByteBuffer.allocate(pageSize + 49);
+
+ target.position(19).limit(19 + pageSize);
+
+ ByteBuffer slicedTarget = target.slice();
+
+ // transfer the segment in chunks into the byte buffer
+ int pos = 0;
+ while (pos < pageSize) {
+ int len = random.nextInt(pageSize / 10);
+ len = Math.min(len, pageSize - pos);
+ seg.get(pos, slicedTarget, len);
+ pos += len;
+ }
+
+ // verify that we wrote the same bytes
+ byte[] result = new byte[pageSize];
+ target.position(19);
+ target.get(result);
+
+ assertArrayEquals(bytes, result);
+ }
+
+ @Test
+ public void testSlicedByteBufferPut() {
+ try {
+ testSlicedByteBufferPut(false);
+ testSlicedByteBufferPut(true);
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ private void testSlicedByteBufferPut(boolean directBuffer) {
+ byte[] bytes = new byte[pageSize + 49];
+ random.nextBytes(bytes);
+
+ ByteBuffer source = directBuffer ?
+ ByteBuffer.allocateDirect(pageSize + 49) :
+ ByteBuffer.allocate(pageSize + 49);
+
+ source.put(bytes);
+ source.position(19).limit(19 + pageSize);
+ ByteBuffer slicedSource = source.slice();
+
+ MemorySegment seg = createSegment(3 * pageSize);
+
+ final int offset = 2 * pageSize;
+
+ // transfer the segment in chunks into the byte buffer
+ int pos = 0;
+ while (pos < pageSize) {
+ int len = random.nextInt(pageSize / 10);
+ len = Math.min(len, pageSize - pos);
+ seg.put(offset + pos, slicedSource, len);
+ pos += len;
+ }
+
+ // verify that we read the same bytes
+ byte[] result = new byte[pageSize];
+ seg.get(offset, result);
+
+ byte[] expected = Arrays.copyOfRange(bytes, 19, 19 + pageSize);
+ assertArrayEquals(expected, result);
+ }
+
+ // ------------------------------------------------------------------------
+ // ByteBuffer overflow / underflow and out of bounds
+ // ------------------------------------------------------------------------
+
+ @Test
+ public void testByteBufferOutOfBounds() {
+ try {
+ final int bbCapacity = pageSize / 10;
+
+ final int[] validOffsets = { 0, 1, pageSize / 10 * 9 };
+ final int[] invalidOffsets = { -1, pageSize + 1, -pageSize, Integer.MAX_VALUE, Integer.MIN_VALUE };
+
+ final int[] validLengths = { 0, 1, bbCapacity, pageSize };
+ final int[] invalidLengths = { -1, -pageSize, Integer.MAX_VALUE, Integer.MIN_VALUE };
+
+ final MemorySegment seg = createSegment(pageSize);
+
+ for (ByteBuffer bb : new ByteBuffer[] {
+ ByteBuffer.allocate(bbCapacity),
+ ByteBuffer.allocateDirect(bbCapacity) } )
+ {
+ for (int off : validOffsets) {
+ for (int len : invalidLengths) {
+ try {
+ seg.put(off, bb, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException | BufferUnderflowException ignored) {}
+
+ try {
+ seg.get(off, bb, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException | BufferOverflowException ignored) {}
+
+ // position/limit may not have changed
+ assertEquals(0, bb.position());
+ assertEquals(bb.capacity(), bb.limit());
+ }
+ }
+
+ for (int off : invalidOffsets) {
+ for (int len : validLengths) {
+ try {
+ seg.put(off, bb, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException | BufferUnderflowException ignored) {}
+
+ try {
+ seg.get(off, bb, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException | BufferOverflowException ignored) {}
+
+ // position/limit may not have changed
+ assertEquals(0, bb.position());
+ assertEquals(bb.capacity(), bb.limit());
+ }
+ }
+
+ for (int off : validOffsets) {
+ for (int len : validLengths) {
+ if (off + len > pageSize) {
+ try {
+ seg.put(off, bb, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException | BufferUnderflowException ignored) {}
+
+ try {
+ seg.get(off, bb, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException | BufferOverflowException ignored) {}
+
+ // position/limit may not have changed
+ assertEquals(0, bb.position());
+ assertEquals(bb.capacity(), bb.limit());
+ }
+ }
+ }
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testByteBufferOverflowUnderflow() {
+ try {
+ final int bbCapacity = pageSize / 10;
+ ByteBuffer bb = ByteBuffer.allocate(bbCapacity);
+
+ MemorySegment seg = createSegment(pageSize);
+
+ try {
+ seg.get(pageSize / 5, bb, pageSize / 10 + 2);
+ fail("should fail with an exception");
+ }
+ catch (BufferOverflowException ignored) {}
+
+ // position / limit should not have been modified
+ assertEquals(0, bb.position());
+ assertEquals(bb.capacity(), bb.limit());
+
+ try {
+ seg.put(pageSize / 5, bb, pageSize / 10 + 2);
+ fail("should fail with an exception");
+ }
+ catch (BufferUnderflowException ignored) {}
+
+ // position / limit should not have been modified
+ assertEquals(0, bb.position());
+ assertEquals(bb.capacity(), bb.limit());
+
+
+ int pos = bb.capacity() / 3;
+ int limit = 2 * bb.capacity() / 3;
+ bb.limit(limit);
+ bb.position(pos);
+
+ try {
+ seg.get(20, bb, bb.capacity() / 3 + 3);
+ fail("should fail with an exception");
+ }
+ catch (BufferOverflowException ignored) {}
+
+ // position / limit should not have been modified
+ assertEquals(pos, bb.position());
+ assertEquals(limit, bb.limit());
+
+ try {
+ seg.put(20, bb, bb.capacity() / 3 + 3);
+ fail("should fail with an exception");
+ }
+ catch (BufferUnderflowException ignored) {}
+
+ // position / limit should not have been modified
+ assertEquals(pos, bb.position());
+ assertEquals(limit, bb.limit());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Comparing and swapping
+ // ------------------------------------------------------------------------
+
+ @Test
+ public void testCompareBytes() {
+ try {
+ final byte[] bytes1 = new byte[pageSize];
+ final byte[] bytes2 = new byte[pageSize];
+
+ final int stride = pageSize / 255;
+ final int shift = 16666;
+
+ for (int i = 0; i < pageSize; i++) {
+ byte val = (byte) ((i / stride) & 0xff);
+ bytes1[i] = val;
+
+ if (i + shift < bytes2.length) {
+ bytes2[i + shift] = val;
+ }
+ }
+
+ MemorySegment seg1 = createSegment(pageSize);
+ MemorySegment seg2 = createSegment(pageSize);
+ seg1.put(0, bytes1);
+ seg2.put(0, bytes2);
+
+ for (int i = 0; i < 1000; i++) {
+ int pos1 = random.nextInt(bytes1.length);
+ int pos2 = random.nextInt(bytes2.length);
+
+ int len = Math.min(Math.min(bytes1.length - pos1, bytes2.length - pos2),
+ random.nextInt(pageSize / 50 ));
+
+ int cmp = seg1.compare(seg2, pos1, pos2, len);
+
+ if (pos1 < pos2 - shift) {
+ assertTrue(cmp <= 0);
+ }
+ else {
+ assertTrue(cmp >= 0);
+ }
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testSwapBytes() {
+ try {
+ final int HALF_SIZE = pageSize / 2;
+
+ final byte[] bytes1 = new byte[pageSize];
+ final byte[] bytes2 = new byte[HALF_SIZE];
+
+ Arrays.fill(bytes2, (byte) 1);
+
+ MemorySegment seg1 = createSegment(pageSize);
+ MemorySegment seg2 = createSegment(HALF_SIZE);
+ seg1.put(0, bytes1);
+ seg2.put(0, bytes2);
+
+ // wap the second half of the first segment with the second segment
+
+ int pos = 0;
+ while (pos < HALF_SIZE) {
+ int len = random.nextInt(pageSize / 40);
+ len = Math.min(len, HALF_SIZE - pos);
+ seg1.swapBytes(new byte[len], seg2, pos + HALF_SIZE, pos, len);
+ pos += len;
+ }
+
+ // the second segment should now be all zeros, the first segment should have one in its second half
+
+ for (int i = 0; i < HALF_SIZE; i++) {
+ assertEquals((byte) 0, seg1.get(i));
+ assertEquals((byte) 0, seg2.get(i));
+ assertEquals((byte) 1, seg1.get(i + HALF_SIZE));
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testCheckAgainstOverflowUnderflowOnRelease() {
+ try {
+ MemorySegment seg = createSegment(512);
+ seg.free();
+
+ // --- bytes (smallest type) ---
+ try {
+ seg.get(0);
+ fail("Expecting an IllegalStateException");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IllegalStateException || e instanceof NullPointerException);
+ }
+
+ try {
+ seg.get(Integer.MAX_VALUE);
+ fail("Expecting an IllegalStateException");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IllegalStateException || e instanceof NullPointerException);
+ }
+
+ try {
+ seg.get(Integer.MIN_VALUE);
+ fail("Expecting an IllegalStateException");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IllegalStateException || e instanceof NullPointerException);
+ }
+
+ // --- longs (largest type) ---
+ try {
+ seg.getLong(0);
+ fail("Expecting an IllegalStateException");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IllegalStateException || e instanceof NullPointerException);
+ }
+
+ try {
+ seg.getLong(Integer.MAX_VALUE);
+ fail("Expecting an IllegalStateException");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IllegalStateException || e instanceof NullPointerException);
+ }
+
+ try {
+ seg.getLong(Integer.MIN_VALUE);
+ fail("Expecting an IllegalStateException");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IllegalStateException || e instanceof NullPointerException);
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Miscellaneous
+ // ------------------------------------------------------------------------
+
+ @Test
+ public void testByteBufferWrapping() {
+ try {
+ MemorySegment seg = createSegment(1024);
+
+ ByteBuffer buf1 = seg.wrap(13, 47);
+ assertEquals(13, buf1.position());
+ assertEquals(60, buf1.limit());
+ assertEquals(47, buf1.remaining());
+
+ ByteBuffer buf2 = seg.wrap(500, 267);
+ assertEquals(500, buf2.position());
+ assertEquals(767, buf2.limit());
+ assertEquals(267, buf2.remaining());
+
+ ByteBuffer buf3 = seg.wrap(0, 1024);
+ assertEquals(0, buf3.position());
+ assertEquals(1024, buf3.limit());
+ assertEquals(1024, buf3.remaining());
+
+ // verify that operations on the byte buffer are correctly reflected
+ // in the memory segment
+ buf3.order(ByteOrder.LITTLE_ENDIAN);
+ buf3.putInt(112, 651797651);
+ assertEquals(651797651, seg.getIntLittleEndian(112));
+
+ buf3.order(ByteOrder.BIG_ENDIAN);
+ buf3.putInt(187, 992288337);
+ assertEquals(992288337, seg.getIntBigEndian(187));
+
+ try {
+ seg.wrap(-1, 20);
+ fail("should throw an exception");
+ }
+ catch (IndexOutOfBoundsException | IllegalArgumentException ignored) {}
+
+ try {
+ seg.wrap(10, -20);
+ fail("should throw an exception");
+ }
+ catch (IndexOutOfBoundsException | IllegalArgumentException ignored) {}
+
+ try {
+ seg.wrap(10, 1024);
+ fail("should throw an exception");
+ }
+ catch (IndexOutOfBoundsException | IllegalArgumentException ignored) {}
+
+ // after freeing, no wrapping should be possible any more.
+ seg.free();
+
+ try {
+ seg.wrap(13, 47);
+ fail("should fail with an exception");
+ }
+ catch (IllegalStateException e) {
+ // expected
+ }
+
+ // existing wraps should stay valid after freeing
+ buf3.order(ByteOrder.LITTLE_ENDIAN);
+ buf3.putInt(112, 651797651);
+ assertEquals(651797651, buf3.getInt(112));
+ buf3.order(ByteOrder.BIG_ENDIAN);
+ buf3.putInt(187, 992288337);
+ assertEquals(992288337, buf3.getInt(187));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testOwner() {
+ try {
+ // a segment without an owner has a null owner
+ assertNull(createSegment(64).getOwner());
+
+ Object theOwner = new Object();
+ MemorySegment seg = createSegment(64, theOwner);
+ assertEquals(theOwner, seg.getOwner());
+
+ // freeing must release the owner, to prevent leaks that prevent class unloading!
+ seg.free();
+ assertNotNull(seg.getOwner());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testSizeAndFreeing() {
+ try {
+ // a segment without an owner has a null owner
+ final int SIZE = 651;
+ MemorySegment seg = createSegment(SIZE);
+
+ assertEquals(SIZE, seg.size());
+ assertFalse(seg.isFreed());
+
+ seg.free();
+ assertTrue(seg.isFreed());
+ assertEquals(SIZE, seg.size());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Parametrization to run with different segment sizes
+ // ------------------------------------------------------------------------
+
+ @Parameterized.Parameters(name = "segment-size = {0}")
+ public static Collection<Object[]> executionModes(){
+ return Arrays.asList(
+ new Object[] { 32*1024 },
+ new Object[] { 4*1024 },
+ new Object[] { 512*1024 } );
+ }
+}
\ No newline at end of file
[14/15] flink git commit: [FLINK-1320] [core] Add an off-heap variant
of the managed memory
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
index ce080d3..31d5563 100644
--- a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
@@ -16,7 +16,6 @@
* limitations under the License.
*/
-
package org.apache.flink.core.memory;
import java.io.DataInput;
@@ -26,26 +25,39 @@ import java.nio.ByteBuffer;
import java.nio.ByteOrder;
/**
- * This class represents a piece of memory allocated from the memory manager. The segment is backed
- * by a byte array and features random put and get methods for the basic types that are stored in a byte-wise
- * fashion in the memory.
- *
+ * This class represents a piece of memory managed by Flink.
+ * The segment may be backed by heap memory (byte array) or by off-heap memory.
+ * <p>
+ * The methods for individual memory access are specialized in the classes
+ * {@link org.apache.flink.core.memory.HeapMemorySegment} and
+ * {@link org.apache.flink.core.memory.HybridMemorySegment}.
+ * All methods that operate across two memory segments are implemented in this class,
+ * to transparently handle the mixing of memory segment types.
* <p>
+ * This class fulfills conceptually a similar purpose as Java's {@link java.nio.ByteBuffer}.
+ * We add this specialized class for various reasons:
+ * <ul>
+ * <li>It offers additional binary compare, swap, and copy methods.</li>
+ * <li>It uses collapsed checks for range check and memory segment disposal.</li>
+ * <li>It offers absolute positioning methods for bulk put/get methods, to guarantee
+ * thread safe use.</li>
+ * <li>It offers explicit big-endian / little-endian access methods, rather than tracking internally
+ * a byte order.</li>
+ * <li>It transparently and efficiently moves data between on-heap and off-heap variants.</li>
+ * </ul>
*
- * Comments on the implementation: We make heavy use of operations that are supported by native
+ * <i>Comments on the implementation</i>:
+ * We make heavy use of operations that are supported by native
* instructions, to achieve a high efficiency. Multi byte types (int, long, float, double, ...)
- * are read and written with "unsafe" native commands. Little-endian to big-endian conversion and
- * vice versa are done using the static <i>reverseBytes</i> methods in the boxing data types
- * (for example {@link Integer#reverseBytes(int)}). On x86/amd64, these are translated by the
- * jit compiler to <i>bswap</i> intrinsic commands.
- *
- * Below is an example of the code generated for the {@link MemorySegment#putLongBigEndian(int, long)}
- * function by the just-in-time compiler. The code is grabbed from an oracle jvm 7 using the
+ * are read and written with "unsafe" native commands.
+ * <p>
+ * Below is an example of the code generated for the {@link HeapMemorySegment#putLongBigEndian(int, long)}
+ * function by the just-in-time compiler. The code is grabbed from an Oracle JVM 7 using the
* hotspot disassembler library (hsdis32.dll) and the jvm command
- * <i>-XX:+UnlockDiagnosticVMOptions -XX:CompileCommand=print,*UnsafeMemorySegment.putLongBigEndian</i>.
+ * <i>-XX:+UnlockDiagnosticVMOptions -XX:CompileCommand=print,*MemorySegment.putLongBigEndian</i>.
* Note that this code realizes both the byte order swapping and the reinterpret cast access to
* get a long from the byte array.
- *
+ *
* <pre>
* [Verified Entry Point]
* 0x00007fc403e19920: sub $0x18,%rsp
@@ -66,286 +78,539 @@ import java.nio.ByteOrder;
* ; {poll_return}
* 0x00007fc403e1994a: retq
* </pre>
+ *
+ * <i>Note on efficiency</i>:
+ * For best efficiency, the code that uses this class should make sure that only one
+ * subclass is loaded, or that the methods that are abstract in this class are used only from one of the
+ * subclasses (either the {@link org.apache.flink.core.memory.HeapMemorySegment}, or the
+ * {@link org.apache.flink.core.memory.HybridMemorySegment}).
+ *
+ * That way, all the abstract methods in the MemorySegment base class have only one loaded
+ * actual implementation. This is easy for the JIT to recognize through class hierarchy analysis,
+ * or by identifying that the invocations are monomorphic (all go to the same concrete
+ * method implementation). Under these conditions, the JIT can perfectly inline methods.
*/
-public class MemorySegment {
+public abstract class MemorySegment {
+
+ /** The unsafe handle for transparent memory copied (heap / off-heap) */
+ @SuppressWarnings("restriction")
+ protected static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+
+ /** The beginning of the byte array contents, relative to the byte array object */
+ @SuppressWarnings("restriction")
+ protected static final long BYTE_ARRAY_BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+
+ /** Constant that flags the byte order. Because this is a boolean constant,
+ * the JIT compiler can use this well to aggressively eliminate the non-applicable code paths */
+ private static final boolean LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+ // ------------------------------------------------------------------------
+
+ /** The heap byte array object relative to which we access the memory. Is non-null if the
+ * memory is on the heap, and is null, if the memory if off the heap. If we have this buffer, we
+ * must never void this reference, or the memory segment will point to undefined addresses
+ * outside the heap and may in out-of-order execution cases cause segmentation faults. */
+ protected final byte[] heapMemory;
+
+ /** The address to the data, relative to the heap memory byte array. If the heap memory byte array
+ * is null, this becomes an absolute memory address outside the heap. */
+ protected long address;
+
+ /** The address one byte after the last addressable byte.
+ * This is address + size while the segment is not disposed */
+ protected final long addressLimit;
- // flag to enable / disable boundary checks. Note that the compiler eliminates the
- // code paths of the checks (as dead code) when this constant is set to false.
- private static final boolean CHECKED = true;
+ /** The size in bytes of the memory segment */
+ protected final int size;
+ /** Optional owner of the memory segment */
+ private final Object owner;
+
/**
- * The array in which the data is stored.
+ * Creates a new memory segment that represents the memory of the byte array.
+ * Since the byte array is backed by on-heap memory, this memory segment holds its
+ * data on heap. The buffer must be at least of size 8 bytes.
+ *
+ * @param buffer The byte array whose memory is represented by this memory segment.
*/
- protected byte[] memory;
-
+ MemorySegment(byte[] buffer, Object owner) {
+ if (buffer == null) {
+ throw new NullPointerException("buffer");
+ }
+
+ this.heapMemory = buffer;
+ this.address = BYTE_ARRAY_BASE_OFFSET;
+ this.size = buffer.length;
+ this.addressLimit = this.address + this.size;
+ this.owner = owner;
+ }
+
/**
- * Wrapper for I/O requests.
+ * Creates a new memory segment that represents the memory at the absolute address given
+ * by the pointer.
+ *
+ * @param offHeapAddress The address of the memory represented by this memory segment.
+ * @param size The size of this memory segment.
*/
- protected ByteBuffer wrapper;
+ MemorySegment(long offHeapAddress, int size, Object owner) {
+ if (offHeapAddress <= 0) {
+ throw new IllegalArgumentException("negative pointer or size");
+ }
+ if (offHeapAddress >= Long.MAX_VALUE - Integer.MAX_VALUE) {
+ // this is necessary to make sure the collapsed checks are safe against numeric overflows
+ throw new IllegalArgumentException("Segment initialized with too large address: " + address
+ + " ; Max allowed address is " + (Long.MAX_VALUE - Integer.MAX_VALUE - 1));
+ }
+
+ this.heapMemory = null;
+ this.address = offHeapAddress;
+ this.addressLimit = this.address + size;
+ this.size = size;
+ this.owner = owner;
+ }
- // -------------------------------------------------------------------------
- // Constructors
- // -------------------------------------------------------------------------
+ // ------------------------------------------------------------------------
+ // Memory Segment Operations
+ // ------------------------------------------------------------------------
/**
- * Creates a new memory segment that represents the data in the given byte array.
- *
- * @param memory The byte array that holds the data.
+ * Gets the size of the memory segment, in bytes.
+ * @return The size of the memory segment.
*/
- public MemorySegment(byte[] memory) {
- this.memory = memory;
+ public int size() {
+ return size;
}
- // -------------------------------------------------------------------------
- // MemorySegment Accessors
- // -------------------------------------------------------------------------
-
/**
- * Checks whether this memory segment has already been freed. In that case, the
- * segment must not be used any more.
- *
- * @return True, if the segment has been freed, false otherwise.
+ * Checks whether the memory segment was freed.
+ * @return True, if the memory segment has been freed, false otherwise.
*/
- public final boolean isFreed() {
- return this.memory == null;
+ public boolean isFreed() {
+ return address > addressLimit;
}
- public final void free() {
- this.wrapper = null;
- this.memory = null;
+ /**
+ * Frees this memory segment. After this operation has been called, no further operations are
+ * possible on the memory segment and will fail. The actual memory (heap or off-heap) will only
+ * be released after this memory segment object has become garbage collected.
+ */
+ public void free() {
+ // this ensures we can place no more data and trigger
+ // the checks for the freed segment
+ address = addressLimit + 1;
}
-
+
/**
- * Gets the size of the memory segment, in bytes. Because segments
- * are backed by arrays, they cannot be larger than two GiBytes.
- *
- * @return The size in bytes.
+ * Checks whether this memory segment is backed by off-heap memory.
+ * @return True, if the memory segment is backed by off-heap memory, false if it is backed
+ * by heap memory.
*/
- public final int size() {
- return this.memory.length;
+ public boolean isOffHeap() {
+ return heapMemory == null;
}
/**
* Wraps the chunk of the underlying memory located between <tt>offset<tt> and
* <tt>length</tt> in a NIO ByteBuffer.
- *
+ *
* @param offset The offset in the memory segment.
* @param length The number of bytes to be wrapped as a buffer.
* @return A <tt>ByteBuffer</tt> backed by the specified portion of the memory segment.
* @throws IndexOutOfBoundsException Thrown, if offset is negative or larger than the memory segment size,
* or if the offset plus the length is larger than the segment size.
*/
- public ByteBuffer wrap(int offset, int length) {
- if (offset > this.memory.length || offset > this.memory.length - length) {
- throw new IndexOutOfBoundsException();
- }
-
- if (this.wrapper == null) {
- this.wrapper = ByteBuffer.wrap(this.memory, offset, length);
- }
- else {
- this.wrapper.limit(offset + length);
- this.wrapper.position(offset);
- }
+ public abstract ByteBuffer wrap(int offset, int length);
- return this.wrapper;
+ /**
+ * Gets the owner of this memory segment. Returns null, if the owner was not set.
+ * @return The owner of the memory segment, or null, if it does not have an owner.
+ */
+ public Object getOwner() {
+ return owner;
}
-
+
+
// ------------------------------------------------------------------------
// Random Access get() and put() methods
// ------------------------------------------------------------------------
- // --------------------------------------------------------------------------------------------
- // WARNING: Any code for range checking must take care to avoid integer overflows. The position
- // integer may go up to <code>Integer.MAX_VALUE</tt>. Range checks that work after the principle
- // <code>position + 3 < end</code> may fail because <code>position + 3</code> becomes negative.
- // A safe solution is to subtract the delta from the limit, for example
- // <code>position < end - 3</code>. Since all indices are always positive, and the integer domain
- // has one more negative value than positive values, this can never cause an underflow.
- // --------------------------------------------------------------------------------------------
-
-
+ //------------------------------------------------------------------------
+ // Notes on the implementation: We try to collapse as many checks as
+ // possible. We need to obey the following rules to make this safe
+ // against segfaults:
+ //
+ // - Grab mutable fields onto the stack before checking and using. This
+ // guards us against concurrent modifications which invalidate the
+ // pointers
+ // - Use subtrations for range checks, as they are tolerant
+ //------------------------------------------------------------------------
+
/**
* Reads the byte at the given position.
- *
+ *
* @param index The position from which the byte will be read
* @return The byte at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger or equal to the size of
* the memory segment.
*/
- public final byte get(int index) {
- return this.memory[index];
- }
+ public abstract byte get(int index);
/**
* Writes the given byte into this buffer at the given position.
- *
+ *
* @param index The index at which the byte will be written.
* @param b The byte value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger or equal to the size of
* the memory segment.
*/
- public final void put(int index, byte b) {
- this.memory[index] = b;
- }
+ public abstract void put(int index, byte b);
/**
* Bulk get method. Copies dst.length memory from the specified position to
* the destination memory.
- *
+ *
* @param index The position at which the first byte will be read.
* @param dst The memory into which the memory will be copied.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or too large that the data between the
* index and the memory segment end is not enough to fill the destination array.
*/
- public final void get(int index, byte[] dst) {
- get(index, dst, 0, dst.length);
- }
+ public abstract void get(int index, byte[] dst);
/**
* Bulk put method. Copies src.length memory from the source memory into the
* memory segment beginning at the specified position.
- *
+ *
* @param index The index in the memory segment array, where the data is put.
* @param src The source array to copy the data from.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or too large such that the array
* size exceed the amount of memory between the index and the memory
* segment's end.
*/
- public final void put(int index, byte[] src) {
- put(index, src, 0, src.length);
- }
+ public abstract void put(int index, byte[] src);
/**
* Bulk get method. Copies length memory from the specified position to the
* destination memory, beginning at the given offset
- *
+ *
* @param index The position at which the first byte will be read.
* @param dst The memory into which the memory will be copied.
* @param offset The copying offset in the destination memory.
* @param length The number of bytes to be copied.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or too large that the requested number of
* bytes exceed the amount of memory between the index and the memory
* segment's end.
*/
- public final void get(int index, byte[] dst, int offset, int length) {
- // system arraycopy does the boundary checks anyways, no need to check extra
- System.arraycopy(this.memory, index, dst, offset, length);
- }
+ public abstract void get(int index, byte[] dst, int offset, int length);
/**
* Bulk put method. Copies length memory starting at position offset from
* the source memory into the memory segment starting at the specified
* index.
- *
+ *
* @param index The position in the memory segment array, where the data is put.
* @param src The source array to copy the data from.
* @param offset The offset in the source array where the copying is started.
* @param length The number of bytes to copy.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or too large such that the array
* portion to copy exceed the amount of memory between the index and the memory
* segment's end.
*/
- public final void put(int index, byte[] src, int offset, int length) {
- // system arraycopy does the boundary checks anyways, no need to check extra
- System.arraycopy(src, offset, this.memory, index, length);
- }
+ public abstract void put(int index, byte[] src, int offset, int length);
/**
* Reads one byte at the given position and returns its boolean
* representation.
- *
+ *
* @param index The position from which the memory will be read.
* @return The boolean value at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 1.
*/
- public final boolean getBoolean(int index) {
- return this.memory[index] != 0;
- }
+ public abstract boolean getBoolean(int index);
/**
* Writes one byte containing the byte value into this buffer at the given
* position.
- *
+ *
* @param index The position at which the memory will be written.
* @param value The char value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 1.
*/
- public final void putBoolean(int index, boolean value) {
- this.memory[index] = (byte) (value ? 1 : 0);
- }
+ public abstract void putBoolean(int index, boolean value);
/**
- * Reads two memory at the given position, composing them into a char value
- * according to the current byte order.
- *
+ * Reads a char value from the given position, in the system's native byte order.
+ *
* @param index The position from which the memory will be read.
* @return The char value at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 2.
*/
+ @SuppressWarnings("restriction")
public final char getChar(int index) {
- return (char) ( ((this.memory[index ] & 0xff) << 8) |
- (this.memory[index + 1] & 0xff) );
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 2) {
+ return UNSAFE.getChar(heapMemory, pos);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
}
/**
- * Writes two memory containing the given char value, in the current byte
- * order, into this buffer at the given position.
- *
+ * Reads an character value (16 bit, 2 bytes) from the given position, in little-endian byte order.
+ * This method's speed depends on the system's native byte order, and it
+ * is possibly slower than {@link #getChar(int)}. For most cases (such as
+ * transient storage in memory or serialization for I/O and network),
+ * it suffices to know that the byte order in which the value is written is the same as the
+ * one in which it is read, and {@link #getChar(int)} is the preferable choice.
+ *
+ * @param index The position from which the value will be read.
+ * @return The character value at the given position.
+ *
+ * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+ */
+ public final char getCharLittleEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return getChar(index);
+ } else {
+ return Character.reverseBytes(getChar(index));
+ }
+ }
+
+ /**
+ * Reads an character value (16 bit, 2 bytes) from the given position, in big-endian byte order.
+ * This method's speed depends on the system's native byte order, and it
+ * is possibly slower than {@link #getChar(int)}. For most cases (such as
+ * transient storage in memory or serialization for I/O and network),
+ * it suffices to know that the byte order in which the value is written is the same as the
+ * one in which it is read, and {@link #getChar(int)} is the preferable choice.
+ *
+ * @param index The position from which the value will be read.
+ * @return The character value at the given position.
+ *
+ * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+ */
+ public final char getCharBigEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return Character.reverseBytes(getChar(index));
+ } else {
+ return getChar(index);
+ }
+ }
+
+ /**
+ * Writes a char value to teh given position, in the system's native byte order.
+ *
* @param index The position at which the memory will be written.
* @param value The char value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 2.
*/
+ @SuppressWarnings("restriction")
public final void putChar(int index, char value) {
- this.memory[index ] = (byte) (value >> 8);
- this.memory[index + 1] = (byte) value;
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 2) {
+ UNSAFE.putChar(heapMemory, pos, value);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("segment has been freed");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ /**
+ * Writes the given character (16 bit, 2 bytes) to the given position in little-endian
+ * byte order. This method's speed depends on the system's native byte order, and it
+ * is possibly slower than {@link #putChar(int, char)}. For most cases (such as
+ * transient storage in memory or serialization for I/O and network),
+ * it suffices to know that the byte order in which the value is written is the same as the
+ * one in which it is read, and {@link #putChar(int, char)} is the preferable choice.
+ *
+ * @param index The position at which the value will be written.
+ * @param value The short value to be written.
+ *
+ * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+ */
+ public final void putCharLittleEndian(int index, char value) {
+ if (LITTLE_ENDIAN) {
+ putChar(index, value);
+ } else {
+ putChar(index, Character.reverseBytes(value));
+ }
+ }
+
+ /**
+ * Writes the given character (16 bit, 2 bytes) to the given position in big-endian
+ * byte order. This method's speed depends on the system's native byte order, and it
+ * is possibly slower than {@link #putChar(int, char)}. For most cases (such as
+ * transient storage in memory or serialization for I/O and network),
+ * it suffices to know that the byte order in which the value is written is the same as the
+ * one in which it is read, and {@link #putChar(int, char)} is the preferable choice.
+ *
+ * @param index The position at which the value will be written.
+ * @param value The short value to be written.
+ *
+ * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+ */
+ public final void putCharBigEndian(int index, char value) {
+ if (LITTLE_ENDIAN) {
+ putChar(index, Character.reverseBytes(value));
+ } else {
+ putChar(index, value);
+ }
}
/**
* Reads two memory at the given position, composing them into a short value
* according to the current byte order.
- *
+ *
* @param index The position from which the memory will be read.
* @return The short value at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 2.
*/
public final short getShort(int index) {
- return (short) (
- ((this.memory[index ] & 0xff) << 8) |
- ((this.memory[index + 1] & 0xff)) );
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 2) {
+ return UNSAFE.getShort(heapMemory, pos);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("segment has been freed");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ /**
+ * Reads an short integer value (16 bit, 2 bytes) from the given position, in little-endian byte order.
+ * This method's speed depends on the system's native byte order, and it
+ * is possibly slower than {@link #getShort(int)}. For most cases (such as
+ * transient storage in memory or serialization for I/O and network),
+ * it suffices to know that the byte order in which the value is written is the same as the
+ * one in which it is read, and {@link #getShort(int)} is the preferable choice.
+ *
+ * @param index The position from which the value will be read.
+ * @return The short value at the given position.
+ *
+ * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+ */
+ public final short getShortLittleEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return getShort(index);
+ } else {
+ return Short.reverseBytes(getShort(index));
+ }
+ }
+
+ /**
+ * Reads an short integer value (16 bit, 2 bytes) from the given position, in big-endian byte order.
+ * This method's speed depends on the system's native byte order, and it
+ * is possibly slower than {@link #getShort(int)}. For most cases (such as
+ * transient storage in memory or serialization for I/O and network),
+ * it suffices to know that the byte order in which the value is written is the same as the
+ * one in which it is read, and {@link #getShort(int)} is the preferable choice.
+ *
+ * @param index The position from which the value will be read.
+ * @return The short value at the given position.
+ *
+ * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+ */
+ public final short getShortBigEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return Short.reverseBytes(getShort(index));
+ } else {
+ return getShort(index);
+ }
}
/**
* Writes the given short value into this buffer at the given position, using
* the native byte order of the system.
- *
+ *
* @param index The position at which the value will be written.
* @param value The short value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 2.
*/
public final void putShort(int index, short value) {
- this.memory[index ] = (byte) (value >> 8);
- this.memory[index + 1] = (byte) value;
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 2) {
+ UNSAFE.putShort(heapMemory, pos, value);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("segment has been freed");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
}
-
+
+ /**
+ * Writes the given short integer value (16 bit, 2 bytes) to the given position in little-endian
+ * byte order. This method's speed depends on the system's native byte order, and it
+ * is possibly slower than {@link #putShort(int, short)}. For most cases (such as
+ * transient storage in memory or serialization for I/O and network),
+ * it suffices to know that the byte order in which the value is written is the same as the
+ * one in which it is read, and {@link #putShort(int, short)} is the preferable choice.
+ *
+ * @param index The position at which the value will be written.
+ * @param value The short value to be written.
+ *
+ * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+ */
+ public final void putShortLittleEndian(int index, short value) {
+ if (LITTLE_ENDIAN) {
+ putShort(index, value);
+ } else {
+ putShort(index, Short.reverseBytes(value));
+ }
+ }
+
+ /**
+ * Writes the given short integer value (16 bit, 2 bytes) to the given position in big-endian
+ * byte order. This method's speed depends on the system's native byte order, and it
+ * is possibly slower than {@link #putShort(int, short)}. For most cases (such as
+ * transient storage in memory or serialization for I/O and network),
+ * it suffices to know that the byte order in which the value is written is the same as the
+ * one in which it is read, and {@link #putShort(int, short)} is the preferable choice.
+ *
+ * @param index The position at which the value will be written.
+ * @param value The short value to be written.
+ *
+ * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+ */
+ public final void putShortBigEndian(int index, short value) {
+ if (LITTLE_ENDIAN) {
+ putShort(index, Short.reverseBytes(value));
+ } else {
+ putShort(index, value);
+ }
+ }
+
/**
* Reads an int value (32bit, 4 bytes) from the given position, in the system's native byte order.
* This method offers the best speed for integer reading and should be used
@@ -353,37 +618,38 @@ public class MemorySegment {
* byte order in which the value is written is the same as the one in which it is read
* (such as transient storage in memory, or serialization for I/O and network), making this
* method the preferable choice.
- *
+ *
* @param index The position from which the value will be read.
* @return The int value at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 4.
*/
- @SuppressWarnings("restriction")
public final int getInt(int index) {
- if (CHECKED) {
- if (index >= 0 && index <= this.memory.length - 4) {
- return UNSAFE.getInt(this.memory, BASE_OFFSET + index);
- } else {
- throw new IndexOutOfBoundsException();
- }
- } else {
- return UNSAFE.getInt(this.memory, BASE_OFFSET + index);
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 4) {
+ return UNSAFE.getInt(heapMemory, pos);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("segment has been freed");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
}
}
-
+
/**
- * Reads an int value (32bit, 4 bytes) from the given position, in little endian byte order.
+ * Reads an int value (32bit, 4 bytes) from the given position, in little-endian byte order.
* This method's speed depends on the system's native byte order, and it
* is possibly slower than {@link #getInt(int)}. For most cases (such as
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #getInt(int)} is the preferable choice.
- *
+ *
* @param index The position from which the value will be read.
* @return The int value at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 4.
*/
@@ -394,18 +660,18 @@ public class MemorySegment {
return Integer.reverseBytes(getInt(index));
}
}
-
+
/**
- * Reads an int value (32bit, 4 bytes) from the given position, in big endian byte order.
+ * Reads an int value (32bit, 4 bytes) from the given position, in big-endian byte order.
* This method's speed depends on the system's native byte order, and it
* is possibly slower than {@link #getInt(int)}. For most cases (such as
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #getInt(int)} is the preferable choice.
- *
+ *
* @param index The position from which the value will be read.
* @return The int value at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 4.
*/
@@ -424,26 +690,27 @@ public class MemorySegment {
* byte order in which the value is written is the same as the one in which it is read
* (such as transient storage in memory, or serialization for I/O and network), making this
* method the preferable choice.
- *
+ *
* @param index The position at which the value will be written.
* @param value The int value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 4.
*/
- @SuppressWarnings("restriction")
public final void putInt(int index, int value) {
- if (CHECKED) {
- if (index >= 0 && index <= this.memory.length - 4) {
- UNSAFE.putInt(this.memory, BASE_OFFSET + index, value);
- } else {
- throw new IndexOutOfBoundsException();
- }
- } else {
- UNSAFE.putInt(this.memory, BASE_OFFSET + index, value);
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 4) {
+ UNSAFE.putInt(heapMemory, pos, value);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("segment has been freed");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
}
}
-
+
/**
* Writes the given int value (32bit, 4 bytes) to the given position in little endian
* byte order. This method's speed depends on the system's native byte order, and it
@@ -451,10 +718,10 @@ public class MemorySegment {
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #putInt(int, int)} is the preferable choice.
- *
+ *
* @param index The position at which the value will be written.
* @param value The int value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 4.
*/
@@ -465,7 +732,7 @@ public class MemorySegment {
putInt(index, Integer.reverseBytes(value));
}
}
-
+
/**
* Writes the given int value (32bit, 4 bytes) to the given position in big endian
* byte order. This method's speed depends on the system's native byte order, and it
@@ -473,10 +740,10 @@ public class MemorySegment {
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #putInt(int, int)} is the preferable choice.
- *
+ *
* @param index The position at which the value will be written.
* @param value The int value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 4.
*/
@@ -487,7 +754,7 @@ public class MemorySegment {
putInt(index, value);
}
}
-
+
/**
* Reads a long value (64bit, 8 bytes) from the given position, in the system's native byte order.
* This method offers the best speed for long integer reading and should be used
@@ -495,26 +762,27 @@ public class MemorySegment {
* byte order in which the value is written is the same as the one in which it is read
* (such as transient storage in memory, or serialization for I/O and network), making this
* method the preferable choice.
- *
+ *
* @param index The position from which the value will be read.
* @return The long value at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
- @SuppressWarnings("restriction")
public final long getLong(int index) {
- if (CHECKED) {
- if (index >= 0 && index <= this.memory.length - 8) {
- return UNSAFE.getLong(this.memory, BASE_OFFSET + index);
- } else {
- throw new IndexOutOfBoundsException();
- }
- } else {
- return UNSAFE.getLong(this.memory, BASE_OFFSET + index);
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 8) {
+ return UNSAFE.getLong(heapMemory, pos);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("segment has been freed");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
}
}
-
+
/**
* Reads a long integer value (64bit, 8 bytes) from the given position, in little endian byte order.
* This method's speed depends on the system's native byte order, and it
@@ -522,10 +790,10 @@ public class MemorySegment {
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #getLong(int)} is the preferable choice.
- *
+ *
* @param index The position from which the value will be read.
* @return The long value at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
@@ -536,7 +804,7 @@ public class MemorySegment {
return Long.reverseBytes(getLong(index));
}
}
-
+
/**
* Reads a long integer value (64bit, 8 bytes) from the given position, in big endian byte order.
* This method's speed depends on the system's native byte order, and it
@@ -544,10 +812,10 @@ public class MemorySegment {
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #getLong(int)} is the preferable choice.
- *
+ *
* @param index The position from which the value will be read.
* @return The long value at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
@@ -566,26 +834,27 @@ public class MemorySegment {
* byte order in which the value is written is the same as the one in which it is read
* (such as transient storage in memory, or serialization for I/O and network), making this
* method the preferable choice.
- *
+ *
* @param index The position at which the value will be written.
* @param value The long value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
- @SuppressWarnings("restriction")
public final void putLong(int index, long value) {
- if (CHECKED) {
- if (index >= 0 && index <= this.memory.length - 8) {
- UNSAFE.putLong(this.memory, BASE_OFFSET + index, value);
- } else {
- throw new IndexOutOfBoundsException();
- }
- } else {
- UNSAFE.putLong(this.memory, BASE_OFFSET + index, value);
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 8) {
+ UNSAFE.putLong(heapMemory, pos, value);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("segment has been freed");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
}
}
-
+
/**
* Writes the given long value (64bit, 8 bytes) to the given position in little endian
* byte order. This method's speed depends on the system's native byte order, and it
@@ -593,10 +862,10 @@ public class MemorySegment {
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #putLong(int, long)} is the preferable choice.
- *
+ *
* @param index The position at which the value will be written.
* @param value The long value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
@@ -607,7 +876,7 @@ public class MemorySegment {
putLong(index, Long.reverseBytes(value));
}
}
-
+
/**
* Writes the given long value (64bit, 8 bytes) to the given position in big endian
* byte order. This method's speed depends on the system's native byte order, and it
@@ -615,10 +884,10 @@ public class MemorySegment {
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #putLong(int, long)} is the preferable choice.
- *
+ *
* @param index The position at which the value will be written.
* @param value The long value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
@@ -629,7 +898,7 @@ public class MemorySegment {
putLong(index, value);
}
}
-
+
/**
* Reads a single-precision floating point value (32bit, 4 bytes) from the given position, in the system's
* native byte order. This method offers the best speed for float reading and should be used
@@ -637,17 +906,17 @@ public class MemorySegment {
* byte order in which the value is written is the same as the one in which it is read
* (such as transient storage in memory, or serialization for I/O and network), making this
* method the preferable choice.
- *
+ *
* @param index The position from which the value will be read.
* @return The float value at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 4.
*/
public final float getFloat(int index) {
return Float.intBitsToFloat(getInt(index));
}
-
+
/**
* Reads a single-precision floating point value (32bit, 4 bytes) from the given position, in little endian
* byte order. This method's speed depends on the system's native byte order, and it
@@ -655,17 +924,17 @@ public class MemorySegment {
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #getFloat(int)} is the preferable choice.
- *
+ *
* @param index The position from which the value will be read.
* @return The long value at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
public final float getFloatLittleEndian(int index) {
return Float.intBitsToFloat(getIntLittleEndian(index));
}
-
+
/**
* Reads a single-precision floating point value (32bit, 4 bytes) from the given position, in big endian
* byte order. This method's speed depends on the system's native byte order, and it
@@ -673,10 +942,10 @@ public class MemorySegment {
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #getFloat(int)} is the preferable choice.
- *
+ *
* @param index The position from which the value will be read.
* @return The long value at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
@@ -691,17 +960,17 @@ public class MemorySegment {
* byte order in which the value is written is the same as the one in which it is read
* (such as transient storage in memory, or serialization for I/O and network), making this
* method the preferable choice.
- *
+ *
* @param index The position at which the value will be written.
* @param value The float value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 4.
*/
public final void putFloat(int index, float value) {
putInt(index, Float.floatToRawIntBits(value));
}
-
+
/**
* Writes the given single-precision float value (32bit, 4 bytes) to the given position in little endian
* byte order. This method's speed depends on the system's native byte order, and it
@@ -709,17 +978,17 @@ public class MemorySegment {
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #putFloat(int, float)} is the preferable choice.
- *
+ *
* @param index The position at which the value will be written.
* @param value The long value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
public final void putFloatLittleEndian(int index, float value) {
putIntLittleEndian(index, Float.floatToRawIntBits(value));
}
-
+
/**
* Writes the given single-precision float value (32bit, 4 bytes) to the given position in big endian
* byte order. This method's speed depends on the system's native byte order, and it
@@ -727,17 +996,17 @@ public class MemorySegment {
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #putFloat(int, float)} is the preferable choice.
- *
+ *
* @param index The position at which the value will be written.
* @param value The long value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
public final void putFloatBigEndian(int index, float value) {
putIntBigEndian(index, Float.floatToRawIntBits(value));
}
-
+
/**
* Reads a double-precision floating point value (64bit, 8 bytes) from the given position, in the system's
* native byte order. This method offers the best speed for double reading and should be used
@@ -745,17 +1014,17 @@ public class MemorySegment {
* byte order in which the value is written is the same as the one in which it is read
* (such as transient storage in memory, or serialization for I/O and network), making this
* method the preferable choice.
- *
+ *
* @param index The position from which the value will be read.
* @return The double value at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
public final double getDouble(int index) {
return Double.longBitsToDouble(getLong(index));
}
-
+
/**
* Reads a double-precision floating point value (64bit, 8 bytes) from the given position, in little endian
* byte order. This method's speed depends on the system's native byte order, and it
@@ -763,17 +1032,17 @@ public class MemorySegment {
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #getDouble(int)} is the preferable choice.
- *
+ *
* @param index The position from which the value will be read.
* @return The long value at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
public final double getDoubleLittleEndian(int index) {
return Double.longBitsToDouble(getLongLittleEndian(index));
}
-
+
/**
* Reads a double-precision floating point value (64bit, 8 bytes) from the given position, in big endian
* byte order. This method's speed depends on the system's native byte order, and it
@@ -781,10 +1050,10 @@ public class MemorySegment {
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #getDouble(int)} is the preferable choice.
- *
+ *
* @param index The position from which the value will be read.
* @return The long value at the given position.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
@@ -799,17 +1068,17 @@ public class MemorySegment {
* byte order in which the value is written is the same as the one in which it is read
* (such as transient storage in memory, or serialization for I/O and network), making this
* method the preferable choice.
- *
+ *
* @param index The position at which the memory will be written.
* @param value The double value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
public final void putDouble(int index, double value) {
putLong(index, Double.doubleToRawLongBits(value));
}
-
+
/**
* Writes the given double-precision floating-point value (64bit, 8 bytes) to the given position in little endian
* byte order. This method's speed depends on the system's native byte order, and it
@@ -817,17 +1086,17 @@ public class MemorySegment {
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #putDouble(int, double)} is the preferable choice.
- *
+ *
* @param index The position at which the value will be written.
* @param value The long value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
public final void putDoubleLittleEndian(int index, double value) {
putLongLittleEndian(index, Double.doubleToRawLongBits(value));
}
-
+
/**
* Writes the given double-precision floating-point value (64bit, 8 bytes) to the given position in big endian
* byte order. This method's speed depends on the system's native byte order, and it
@@ -835,70 +1104,53 @@ public class MemorySegment {
* transient storage in memory or serialization for I/O and network),
* it suffices to know that the byte order in which the value is written is the same as the
* one in which it is read, and {@link #putDouble(int, double)} is the preferable choice.
- *
+ *
* @param index The position at which the value will be written.
* @param value The long value to be written.
- *
+ *
* @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
* size minus 8.
*/
public final void putDoubleBigEndian(int index, double value) {
putLongBigEndian(index, Double.doubleToRawLongBits(value));
}
-
+
// -------------------------------------------------------------------------
// Bulk Read and Write Methods
// -------------------------------------------------------------------------
-
- /**
- * Bulk get method. Copies length memory from the specified offset to the
- * provided <tt>DataOutput</tt>.
- *
- * @param out The data output object to copy the data to.
- * @param offset The first byte to by copied.
- * @param length The number of bytes to copy.
- *
- * @throws IOException Thrown, if the DataOutput encountered a problem upon writing.
- */
- public final void get(DataOutput out, int offset, int length) throws IOException {
- out.write(this.memory, offset, length);
- }
+
+ public abstract void get(DataOutput out, int offset, int length) throws IOException;
/**
* Bulk put method. Copies length memory from the given DataInput to the
* memory starting at position offset.
- *
+ *
* @param in The DataInput to get the data from.
* @param offset The position in the memory segment to copy the chunk to.
* @param length The number of bytes to get.
- *
+ *
* @throws IOException Thrown, if the DataInput encountered a problem upon reading,
* such as an End-Of-File.
*/
- public final void put(DataInput in, int offset, int length) throws IOException {
- in.readFully(this.memory, offset, length);
- }
-
+ public abstract void put(DataInput in, int offset, int length) throws IOException;
+
/**
* Bulk get method. Copies {@code numBytes} bytes from this memory segment, starting at position
* {@code offset} to the target {@code ByteBuffer}. The bytes will be put into the target buffer
* starting at the buffer's current position. If this method attempts to write more bytes than
* the target byte buffer has remaining (with respect to {@link ByteBuffer#remaining()}),
* this method will cause a {@link java.nio.BufferOverflowException}.
- *
+ *
* @param offset The position where the bytes are started to be read from in this memory segment.
* @param target The ByteBuffer to copy the bytes to.
* @param numBytes The number of bytes to copy.
- *
+ *
* @throws IndexOutOfBoundsException If the offset is invalid, or this segment does not
* contain the given number of bytes (starting from offset), or the target byte buffer does
* not have enough space for the bytes.
*/
- public final void get(int offset, ByteBuffer target, int numBytes) {
- // ByteBuffer performs the boundy checks
- target.put(this.memory, offset, numBytes);
- }
-
+ public abstract void get(int offset, ByteBuffer target, int numBytes);
+
/**
* Bulk put method. Copies {@code numBytes} bytes from the given {@code ByteBuffer}, into
* this memory segment. The bytes will be read from the target buffer
@@ -907,68 +1159,132 @@ public class MemorySegment {
* If this method attempts to read more bytes than
* the target byte buffer has remaining (with respect to {@link ByteBuffer#remaining()}),
* this method will cause a {@link java.nio.BufferUnderflowException}.
- *
+ *
* @param offset The position where the bytes are started to be written to in this memory segment.
* @param source The ByteBuffer to copy the bytes from.
* @param numBytes The number of bytes to copy.
- *
+ *
* @throws IndexOutOfBoundsException If the offset is invalid, or the source buffer does not
* contain the given number of bytes, or this segment does
* not have enough space for the bytes (counting from offset).
*/
- public final void put(int offset, ByteBuffer source, int numBytes) {
- // ByteBuffer performs the boundy checks
- source.get(this.memory, offset, numBytes);
- }
-
+ public abstract void put(int offset, ByteBuffer source, int numBytes);
+
/**
* Bulk copy method. Copies {@code numBytes} bytes from this memory segment, starting at position
* {@code offset} to the target memory segment. The bytes will be put into the target segment
* starting at position {@code targetOffset}.
- *
+ *
* @param offset The position where the bytes are started to be read from in this memory segment.
* @param target The memory segment to copy the bytes to.
* @param targetOffset The position in the target memory segment to copy the chunk to.
* @param numBytes The number of bytes to copy.
- *
+ *
* @throws IndexOutOfBoundsException If either of the offsets is invalid, or the source segment does not
* contain the given number of bytes (starting from offset), or the target segment does
* not have enough space for the bytes (counting from targetOffset).
*/
public final void copyTo(int offset, MemorySegment target, int targetOffset, int numBytes) {
- // system arraycopy does the boundary checks anyways, no need to check extra
- System.arraycopy(this.memory, offset, target.memory, targetOffset, numBytes);
+ final byte[] thisHeapRef = this.heapMemory;
+ final byte[] otherHeapRef = target.heapMemory;
+ final long thisPointer = this.address + offset;
+ final long otherPointer = target.address + targetOffset;
+
+ if ( (numBytes | offset | targetOffset) >= 0 &&
+ thisPointer <= this.addressLimit - numBytes && otherPointer <= target.addressLimit - numBytes)
+ {
+ UNSAFE.copyMemory(thisHeapRef, thisPointer, otherHeapRef, otherPointer, numBytes);
+ }
+ else if (this.address > this.addressLimit) {
+ throw new IllegalStateException("this memory segment has been freed.");
+ }
+ else if (target.address > target.addressLimit) {
+ throw new IllegalStateException("target memory segment has been freed.");
+ }
+ else {
+ throw new IndexOutOfBoundsException(
+ String.format("offset=%d, targetOffset=%d, numBytes=%d, address=%d, targetAddress=%d",
+ offset, targetOffset, numBytes, this.address, target.address));
+ }
}
-
+
// -------------------------------------------------------------------------
// Comparisons & Swapping
// -------------------------------------------------------------------------
-
- public static final int compare(MemorySegment seg1, MemorySegment seg2, int offset1, int offset2, int len) {
- final byte[] b1 = seg1.memory;
- final byte[] b2 = seg2.memory;
-
- int val = 0;
- for (int pos = 0; pos < len && (val = (b1[offset1 + pos] & 0xff) - (b2[offset2 + pos] & 0xff)) == 0; pos++);
- return val;
- }
-
- public static final void swapBytes(MemorySegment seg1, MemorySegment seg2, byte[] tempBuffer, int offset1, int offset2, int len) {
- // system arraycopy does the boundary checks anyways, no need to check extra
- System.arraycopy(seg1.memory, offset1, tempBuffer, 0, len);
- System.arraycopy(seg2.memory, offset2, seg1.memory, offset1, len);
- System.arraycopy(tempBuffer, 0, seg2.memory, offset2, len);
+
+ /**
+ * Compares two memory segment regions.
+ *
+ * @param seg2 Segment to compare this segment with
+ * @param offset1 Offset of this segment to start comparing
+ * @param offset2 Offset of seg2 to start comparing
+ * @param len Length of the compared memory region
+ *
+ * @return 0 if equal, -1 if seg1 < seg2, 1 otherwise
+ */
+ public final int compare(MemorySegment seg2, int offset1, int offset2, int len) {
+ while (len >= 8) {
+ long l1 = this.getLongBigEndian(offset1);
+ long l2 = seg2.getLongBigEndian(offset2);
+
+ if (l1 != l2) {
+ return (l1 < l2) ^ (l1 < 0) ^ (l2 < 0) ? -1 : 1;
+ }
+
+ offset1 += 8;
+ offset2 += 8;
+ len -= 8;
+ }
+ while (len > 0) {
+ int b1 = this.get(offset1) & 0xff;
+ int b2 = seg2.get(offset2) & 0xff;
+ int cmp = b1 - b2;
+ if (cmp != 0) {
+ return cmp;
+ }
+ offset1++;
+ offset2++;
+ len--;
+ }
+ return 0;
}
+
+ /**
+ * Swaps bytes between two memory segments, using the given auxiliary buffer.
+ *
+ * @param tempBuffer The auxiliary buffer in which to put data during triangle swap.
+ * @param seg2 Segment to swap bytes with
+ * @param offset1 Offset of this segment to start swapping
+ * @param offset2 Offset of seg2 to start swapping
+ * @param len Length of the swapped memory region
+ */
+ public final void swapBytes(byte[] tempBuffer, MemorySegment seg2, int offset1, int offset2, int len) {
+ if ( (offset1 | offset2 | len | (tempBuffer.length - len) ) >= 0) {
+ final long thisPos = this.address + offset1;
+ final long otherPos = seg2.address + offset2;
+
+ if (thisPos <= this.addressLimit - len && otherPos <= seg2.addressLimit - len) {
+ // this -> temp buffer
+ UNSAFE.copyMemory(this.heapMemory, thisPos, tempBuffer, BYTE_ARRAY_BASE_OFFSET, len);
- // --------------------------------------------------------------------------------------------
- // Utilities for native memory accesses and checks
- // --------------------------------------------------------------------------------------------
-
- @SuppressWarnings("restriction")
- private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
-
- @SuppressWarnings("restriction")
- private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+ // other -> this
+ UNSAFE.copyMemory(seg2.heapMemory, otherPos, this.heapMemory, thisPos, len);
- private static final boolean LITTLE_ENDIAN = (MemoryUtils.NATIVE_BYTE_ORDER == ByteOrder.LITTLE_ENDIAN);
+ // temp buffer -> other
+ UNSAFE.copyMemory(tempBuffer, BYTE_ARRAY_BASE_OFFSET, seg2.heapMemory, otherPos, len);
+ return;
+ }
+ else if (this.address > this.addressLimit) {
+ throw new IllegalStateException("this memory segment has been freed.");
+ }
+ else if (seg2.address > seg2.addressLimit) {
+ throw new IllegalStateException("other memory segment has been freed.");
+ }
+ }
+
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException(
+ String.format("offset1=%d, offset2=%d, len=%d, bufferSize=%d, address1=%d, address2=%d",
+ offset1, offset2, len, tempBuffer.length, this.address, seg2.address));
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java
new file mode 100644
index 0000000..0e4e469
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java
@@ -0,0 +1,211 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A factory for memory segments. The purpose of this factory is to make sure that all memory segments
+ * for heap data are of the same type. That way, the runtime does not mix the various specializations
+ * of the {@link org.apache.flink.core.memory.MemorySegment}. Not mixing them has shown to be beneficial
+ * to method specialization by the JIT and to overall performance.
+ * <p>
+ * Note that this factory auto-initialized to use {@link org.apache.flink.core.memory.HeapMemorySegment},
+ * if a request to create a segment comes before the initialization.
+ */
+public class MemorySegmentFactory {
+
+ /** The factory to use */
+ private static volatile Factory factory;
+
+ /**
+ * Creates a new memory segment that targets the given heap memory region.
+ * This method should be used to turn short lived byte arrays into memory segments.
+ *
+ * @param buffer The heap memory region.
+ * @return A new memory segment that targets the given heap memory region.
+ */
+ public static MemorySegment wrap(byte[] buffer) {
+ ensureInitialized();
+ return factory.wrap(buffer);
+ }
+
+ /**
+ * Allocates some unpooled memory and creates a new memory segment that represents
+ * that memory.
+ * <p>
+ * This method is similar to {@link #allocateUnpooledSegment(int, Object)}, but the
+ * memory segment will have null as the owner.
+ *
+ * @param size The size of the memory segment to allocate.
+ * @return A new memory segment, backed by unpooled heap memory.
+ */
+ public static MemorySegment allocateUnpooledSegment(int size) {
+ return allocateUnpooledSegment(size, null);
+ }
+
+ /**
+ * Allocates some unpooled memory and creates a new memory segment that represents
+ * that memory.
+ * <p>
+ * This method is similar to {@link #allocateUnpooledSegment(int)}, but additionally sets
+ * the owner of the memory segment.
+ *
+ * @param size The size of the memory segment to allocate.
+ * @param owner The owner to associate with the memory segment.
+ * @return A new memory segment, backed by unpooled heap memory.
+ */
+ public static MemorySegment allocateUnpooledSegment(int size, Object owner) {
+ ensureInitialized();
+ return factory.allocateUnpooledSegment(size, owner);
+ }
+
+ /**
+ * Creates a memory segment that wraps the given byte array.
+ * <p>
+ * This method is intended to be used for components which pool memory and create
+ * memory segments around long-lived memory regions.
+ *
+ *
+ * @param memory The heap memory to be represented by the memory segment.
+ * @param owner The owner to associate with the memory segment.
+ * @return A new memory segment representing the given heap memory.
+ */
+ public static MemorySegment wrapPooledHeapMemory(byte[] memory, Object owner) {
+ ensureInitialized();
+ return factory.wrapPooledHeapMemory(memory, owner);
+ }
+
+ /**
+ * Creates a memory segment that wraps the off-heap memory backing the given ByteBuffer.
+ * Note that the ByteBuffer needs to be a <i>direct ByteBuffer</i>.
+ * <p>
+ * This method is intended to be used for components which pool memory and create
+ * memory segments around long-lived memory regions.
+ *
+ * @param memory The byte buffer with the off-heap memory to be represented by the memory segment.
+ * @param owner The owner to associate with the memory segment.
+ * @return A new memory segment representing the given off-heap memory.
+ */
+ public static MemorySegment wrapPooledOffHeapMemory(ByteBuffer memory, Object owner) {
+ ensureInitialized();
+ return factory.wrapPooledOffHeapMemory(memory, owner);
+ }
+
+ // ------------------------------------------------------------------------
+
+ /**
+ * Initializes this factory with the given concrete factory.
+ *
+ * @param f The concrete factory to use.
+ * @throws java.lang.IllegalStateException Thrown, if this factory has been initialized before.
+ */
+ public static void initializeFactory(Factory f) {
+ if (f == null) {
+ throw new NullPointerException();
+ }
+
+ synchronized (MemorySegmentFactory.class) {
+ if (factory == null) {
+ factory = f;
+ }
+ else {
+ throw new IllegalStateException("Factory has already been initialized");
+ }
+ }
+ }
+
+ /**
+ * Checks whether this memory segment factory has been initialized (with a type to produce).
+ *
+ * @return True, if the factory has been initialized, false otherwise.
+ */
+ public static boolean isInitialized() {
+ return factory != null;
+ }
+
+ /**
+ * Gets the factory. May return null, if the factory has not been initialized.
+ *
+ * @return The factory, or null, if the factory has not been initialized.
+ */
+ public static Factory getFactory() {
+ return factory;
+ }
+
+ private static void ensureInitialized() {
+ if (factory == null) {
+ factory = HeapMemorySegment.FACTORY;
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Internal factory
+ // ------------------------------------------------------------------------
+
+ /**
+ * A concrete factory for memory segments.
+ */
+ public static interface Factory {
+
+ /**
+ * Creates a new memory segment that targets the given heap memory region.
+ *
+ * @param memory The heap memory region.
+ * @return A new memory segment that targets the given heap memory region.
+ */
+ MemorySegment wrap(byte[] memory);
+
+ /**
+ * Allocates some unpooled memory and creates a new memory segment that represents
+ * that memory.
+ *
+ * @param size The size of the memory segment to allocate.
+ * @param owner The owner to associate with the memory segment.
+ * @return A new memory segment, backed by unpooled heap memory.
+ */
+ MemorySegment allocateUnpooledSegment(int size, Object owner);
+
+ /**
+ * Creates a memory segment that wraps the given byte array.
+ * <p>
+ * This method is intended to be used for components which pool memory and create
+ * memory segments around long-lived memory regions.
+ *
+ *
+ * @param memory The heap memory to be represented by the memory segment.
+ * @param owner The owner to associate with the memory segment.
+ * @return A new memory segment representing the given heap memory.
+ */
+ MemorySegment wrapPooledHeapMemory(byte[] memory, Object owner);
+
+ /**
+ * Creates a memory segment that wraps the off-heap memory backing the given ByteBuffer.
+ * Note that the ByteBuffer needs to be a <i>direct ByteBuffer</i>.
+ * <p>
+ * This method is intended to be used for components which pool memory and create
+ * memory segments around long-lived memory regions.
+ *
+ * @param memory The byte buffer with the off-heap memory to be represented by the memory segment.
+ * @param owner The owner to associate with the memory segment.
+ * @return A new memory segment representing the given off-heap memory.
+ */
+ MemorySegment wrapPooledOffHeapMemory(ByteBuffer memory, Object owner);
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/main/java/org/apache/flink/core/memory/MemoryType.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemoryType.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryType.java
new file mode 100644
index 0000000..5b2b5d3
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryType.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory;
+
+/**
+ * The class of memory, such as heap or off-heap.
+ */
+public enum MemoryType {
+
+ /**
+ * Denotes memory that is part of the Java heap.
+ */
+ HEAP,
+
+ /**
+ * Denotes memory that is outside the Java heap (but still part of tha Java process).
+ */
+ OFF_HEAP
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java
index c1f626f..20b37c9 100644
--- a/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java
@@ -22,22 +22,17 @@ import java.lang.reflect.Field;
import java.nio.ByteOrder;
/**
- * Utility class for native (unsafe) memory accesses.
+ * Utility class for memory operations.
*/
public class MemoryUtils {
- /**
- * The "unsafe", which can be used to perform native memory accesses.
- */
+ /** The "unsafe", which can be used to perform native memory accesses. */
@SuppressWarnings("restriction")
public static final sun.misc.Unsafe UNSAFE = getUnsafe();
- /**
- * The native byte order of the platform on which the system currently runs.
- */
- public static final ByteOrder NATIVE_BYTE_ORDER = getByteOrder();
-
-
+ /** The native byte order of the platform on which the system currently runs. */
+ public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
@SuppressWarnings("restriction")
private static sun.misc.Unsafe getUnsafe() {
try {
@@ -45,21 +40,18 @@ public class MemoryUtils {
unsafeField.setAccessible(true);
return (sun.misc.Unsafe) unsafeField.get(null);
} catch (SecurityException e) {
- throw new RuntimeException("Could not access the unsafe handle.", e);
+ throw new RuntimeException("Could not access the sun.misc.Unsafe handle, permission denied by security manager.", e);
} catch (NoSuchFieldException e) {
- throw new RuntimeException("The static unsafe handle field was not be found.");
+ throw new RuntimeException("The static handle field in sun.misc.Unsafe was not found.");
} catch (IllegalArgumentException e) {
- throw new RuntimeException("Bug: Illegal argument reflection access for static field.");
+ throw new RuntimeException("Bug: Illegal argument reflection access for static field.", e);
} catch (IllegalAccessException e) {
- throw new RuntimeException("Access to the unsafe handle is forbidden by the runtime.", e);
+ throw new RuntimeException("Access to sun.misc.Unsafe is forbidden by the runtime.", e);
+ } catch (Throwable t) {
+ throw new RuntimeException("Unclassified error while trying to access the sun.misc.Unsafe handle.", t);
}
}
-
- @SuppressWarnings("restriction")
- private static ByteOrder getByteOrder() {
- return ByteOrder.nativeOrder();
- }
-
-
+
+ /** Should not be instantiated */
private MemoryUtils() {}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java
index 782c402..a8ace92 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java
@@ -29,7 +29,9 @@ import static org.junit.Assert.*;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.util.TestLogger;
+
import org.junit.Assert;
import org.junit.Test;
@@ -238,7 +240,7 @@ public abstract class ComparatorTestBase<T> extends TestLogger {
// Help Function for setting up a memory segment and normalize the keys of the data array in it
public MemorySegment setupNormalizedKeysMemSegment(T[] data, int normKeyLen, TypeComparator<T> comparator) {
- MemorySegment memSeg = new MemorySegment(new byte[2048]);
+ MemorySegment memSeg = MemorySegmentFactory.allocateUnpooledSegment(2048);
// Setup normalized Keys in the memory segment
int offset = 0;
@@ -294,7 +296,7 @@ public abstract class ComparatorTestBase<T> extends TestLogger {
MemorySegment memSeg2 = setupNormalizedKeysMemSegment(data, normKeyLen, comparator);
for (int i = 0; i < data.length; i++) {
- assertTrue(MemorySegment.compare(memSeg1, memSeg2, i * normKeyLen, i * normKeyLen, normKeyLen) == 0);
+ assertTrue(memSeg1.compare(memSeg2, i * normKeyLen, i * normKeyLen, normKeyLen) == 0);
}
} catch (Exception e) {
System.err.println(e.getMessage());
@@ -343,14 +345,14 @@ public abstract class ComparatorTestBase<T> extends TestLogger {
for (int h = l + 1; h < data.length; h++) {
int cmp;
if (greater) {
- cmp = MemorySegment.compare(memSegLow, memSegHigh, l * normKeyLen, h * normKeyLen, normKeyLen);
+ cmp = memSegLow.compare(memSegHigh, l * normKeyLen, h * normKeyLen, normKeyLen);
if (fullyDetermines) {
assertTrue(cmp < 0);
} else {
assertTrue(cmp <= 0);
}
} else {
- cmp = MemorySegment.compare(memSegHigh, memSegLow, h * normKeyLen, l * normKeyLen, normKeyLen);
+ cmp = memSegHigh.compare(memSegLow, h * normKeyLen, l * normKeyLen, normKeyLen);
if (fullyDetermines) {
assertTrue(cmp > 0);
} else {
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java
new file mode 100644
index 0000000..724a366
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java
@@ -0,0 +1,356 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory;
+
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Random;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class CrossSegmentTypeTest {
+
+ private final int pageSize = 32*1024;
+
+ // ------------------------------------------------------------------------
+
+ @Test
+ public void testCompareBytesMixedSegments() {
+ try {
+ MemorySegment[] segs1 = {
+ new HeapMemorySegment(new byte[pageSize]),
+ new HybridMemorySegment(new byte[pageSize]),
+ new HybridMemorySegment(ByteBuffer.allocateDirect(pageSize))
+ };
+
+ MemorySegment[] segs2 = {
+ new HeapMemorySegment(new byte[pageSize]),
+ new HybridMemorySegment(new byte[pageSize]),
+ new HybridMemorySegment(ByteBuffer.allocateDirect(pageSize))
+ };
+
+ Random rnd = new Random();
+
+ for (MemorySegment seg1 : segs1) {
+ for (MemorySegment seg2 : segs2) {
+ testCompare(seg1, seg2, rnd);
+ }
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ private void testCompare(MemorySegment seg1, MemorySegment seg2, Random random) {
+ assertEquals(pageSize, seg1.size());
+ assertEquals(pageSize, seg2.size());
+
+ final byte[] bytes1 = new byte[pageSize];
+ final byte[] bytes2 = new byte[pageSize];
+
+ final int stride = pageSize / 255;
+ final int shift = 16666;
+
+ for (int i = 0; i < pageSize; i++) {
+ byte val = (byte) ((i / stride) & 0xff);
+ bytes1[i] = val;
+
+ if (i + shift < bytes2.length) {
+ bytes2[i + shift] = val;
+ }
+ }
+
+ seg1.put(0, bytes1);
+ seg2.put(0, bytes2);
+
+ for (int i = 0; i < 1000; i++) {
+ int pos1 = random.nextInt(bytes1.length);
+ int pos2 = random.nextInt(bytes2.length);
+
+ int len = Math.min(Math.min(bytes1.length - pos1, bytes2.length - pos2),
+ random.nextInt(pageSize / 50 ));
+
+ int cmp = seg1.compare(seg2, pos1, pos2, len);
+
+ if (pos1 < pos2 - shift) {
+ assertTrue(cmp <= 0);
+ }
+ else {
+ assertTrue(cmp >= 0);
+ }
+ }
+ }
+
+
+ @Test
+ public void testSwapBytesMixedSegments() {
+ try {
+ final int HALF_SIZE = pageSize / 2;
+
+ MemorySegment[] segs1 = {
+ new HeapMemorySegment(new byte[pageSize]),
+ new HybridMemorySegment(new byte[pageSize]),
+ new HybridMemorySegment(ByteBuffer.allocateDirect(pageSize))
+ };
+
+ MemorySegment[] segs2 = {
+ new HeapMemorySegment(new byte[HALF_SIZE]),
+ new HybridMemorySegment(new byte[HALF_SIZE]),
+ new HybridMemorySegment(ByteBuffer.allocateDirect(HALF_SIZE))
+ };
+
+ Random rnd = new Random();
+
+ for (MemorySegment seg1 : segs1) {
+ for (MemorySegment seg2 : segs2) {
+ testSwap(seg1, seg2, rnd, HALF_SIZE);
+ }
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ private void testSwap(MemorySegment seg1, MemorySegment seg2, Random random, int smallerSize) {
+ assertEquals(pageSize, seg1.size());
+ assertEquals(smallerSize, seg2.size());
+
+ final byte[] bytes1 = new byte[pageSize];
+ final byte[] bytes2 = new byte[smallerSize];
+
+ Arrays.fill(bytes2, (byte) 1);
+
+ seg1.put(0, bytes1);
+ seg2.put(0, bytes2);
+
+ // wap the second half of the first segment with the second segment
+
+ int pos = 0;
+ while (pos < smallerSize) {
+ int len = random.nextInt(pageSize / 40);
+ len = Math.min(len, smallerSize - pos);
+ seg1.swapBytes(new byte[len], seg2, pos + smallerSize, pos, len);
+ pos += len;
+ }
+
+ // the second segment should now be all zeros, the first segment should have one in its second half
+
+ for (int i = 0; i < smallerSize; i++) {
+ assertEquals((byte) 0, seg1.get(i));
+ assertEquals((byte) 0, seg2.get(i));
+ assertEquals((byte) 1, seg1.get(i + smallerSize));
+ }
+ }
+
+ @Test
+ public void testCopyMixedSegments() {
+ try {
+ MemorySegment[] segs1 = {
+ new HeapMemorySegment(new byte[pageSize]),
+ new HybridMemorySegment(new byte[pageSize]),
+ new HybridMemorySegment(ByteBuffer.allocateDirect(pageSize))
+ };
+
+ MemorySegment[] segs2 = {
+ new HeapMemorySegment(new byte[pageSize]),
+ new HybridMemorySegment(new byte[pageSize]),
+ new HybridMemorySegment(ByteBuffer.allocateDirect(pageSize))
+ };
+
+ Random rnd = new Random();
+
+ for (MemorySegment seg1 : segs1) {
+ for (MemorySegment seg2 : segs2) {
+ testCopy(seg1, seg2, rnd);
+ }
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ private void testCopy(MemorySegment seg1, MemorySegment seg2, Random random) {
+ assertEquals(pageSize, seg1.size());
+ assertEquals(pageSize, seg2.size());
+
+ byte[] expected = new byte[pageSize];
+ byte[] actual = new byte[pageSize];
+
+ // zero out the memory
+ seg1.put(0, expected);
+ seg2.put(0, expected);
+
+ for (int i = 0; i < 40; i++) {
+ int numBytes = random.nextInt(pageSize / 20);
+ byte[] bytes = new byte[numBytes];
+ random.nextBytes(bytes);
+
+ int thisPos = random.nextInt(pageSize - numBytes);
+ int otherPos = random.nextInt(pageSize - numBytes);
+
+ // track what we expect
+ System.arraycopy(bytes, 0, expected, otherPos, numBytes);
+
+ seg1.put(thisPos, bytes);
+ seg1.copyTo(thisPos, seg2, otherPos, numBytes);
+ }
+
+ seg2.get(0, actual);
+ assertArrayEquals(expected, actual);
+
+ // test out of bound conditions
+
+ final int[] validOffsets = { 0, 1, pageSize / 10 * 9 };
+ final int[] invalidOffsets = { -1, pageSize + 1, -pageSize, Integer.MAX_VALUE, Integer.MIN_VALUE };
+
+ final int[] validLengths = { 0, 1, pageSize / 10, pageSize };
+ final int[] invalidLengths = { -1, -pageSize, pageSize + 1, Integer.MAX_VALUE, Integer.MIN_VALUE };
+
+ for (int off1 : validOffsets) {
+ for (int off2 : validOffsets) {
+ for (int len : invalidLengths) {
+ try {
+ seg1.copyTo(off1, seg2, off2, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+
+ try {
+ seg1.copyTo(off2, seg2, off1, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+
+ try {
+ seg2.copyTo(off1, seg1, off2, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+
+ try {
+ seg2.copyTo(off2, seg1, off1, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+ }
+ }
+ }
+
+ for (int off1 : validOffsets) {
+ for (int off2 : invalidOffsets) {
+ for (int len : validLengths) {
+ try {
+ seg1.copyTo(off1, seg2, off2, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+
+ try {
+ seg1.copyTo(off2, seg2, off1, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+
+ try {
+ seg2.copyTo(off1, seg1, off2, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+
+ try {
+ seg2.copyTo(off2, seg1, off1, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+ }
+ }
+ }
+
+ for (int off1 : invalidOffsets) {
+ for (int off2 : validOffsets) {
+ for (int len : validLengths) {
+ try {
+ seg1.copyTo(off1, seg2, off2, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+
+ try {
+ seg1.copyTo(off2, seg2, off1, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+
+ try {
+ seg2.copyTo(off1, seg1, off2, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+
+ try {
+ seg2.copyTo(off2, seg1, off1, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+ }
+ }
+ }
+
+ for (int off1 : invalidOffsets) {
+ for (int off2 : invalidOffsets) {
+ for (int len : validLengths) {
+ try {
+ seg1.copyTo(off1, seg2, off2, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+
+ try {
+ seg1.copyTo(off2, seg2, off1, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+
+ try {
+ seg2.copyTo(off1, seg1, off2, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+
+ try {
+ seg2.copyTo(off2, seg1, off1, len);
+ fail("should fail with an IndexOutOfBoundsException");
+ }
+ catch (IndexOutOfBoundsException ignored) {}
+ }
+ }
+ }
+ }
+}
[03/15] flink git commit: [FLINK-2635] [streaming] Make input
processors independent of batch reader interface.
Posted by se...@apache.org.
[FLINK-2635] [streaming] Make input processors independent of batch reader interface.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/b18e410b
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/b18e410b
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/b18e410b
Branch: refs/heads/master
Commit: b18e410bc336eb46497aaa75633c16ebe6139554
Parents: c09d14a
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Sep 8 16:19:24 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Sep 8 20:58:05 2015 +0200
----------------------------------------------------------------------
.../runtime/io/StreamInputProcessor.java | 22 +++++++++--------
.../runtime/io/StreamTwoInputProcessor.java | 25 +++++++-------------
.../flink/streaming/api/PartitionerTest.java | 7 +++---
3 files changed, 25 insertions(+), 29 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/b18e410b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
index cc91d63..8ce8a01 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
import org.apache.flink.runtime.event.AbstractEvent;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.network.api.reader.AbstractReader;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult;
import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer;
@@ -52,19 +52,21 @@ import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
*
* @param <IN> The type of the record that can be read with this record reader.
*/
-public class StreamInputProcessor<IN> extends AbstractReader implements StreamingReader {
+public class StreamInputProcessor<IN> {
private final RecordDeserializer<DeserializationDelegate<StreamElement>>[] recordDeserializers;
private RecordDeserializer<DeserializationDelegate<StreamElement>> currentRecordDeserializer;
+ private final CheckpointBarrierHandler barrierHandler;
+
// We need to keep track of the channel from which a buffer came, so that we can
// appropriately map the watermarks to input channels
private int currentChannel = -1;
private boolean isFinished;
- private final CheckpointBarrierHandler barrierHandler;
+
private final long[] watermarks;
private long lastEmittedWatermark;
@@ -77,8 +79,8 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
CheckpointingMode checkpointMode,
IOManager ioManager,
boolean enableWatermarkMultiplexing) throws IOException {
-
- super(InputGateUtil.createInputGate(inputGates));
+
+ InputGate inputGate = InputGateUtil.createInputGate(inputGates);
if (checkpointMode == CheckpointingMode.EXACTLY_ONCE) {
this.barrierHandler = new BarrierBuffer(inputGate, ioManager);
@@ -173,7 +175,9 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
else {
// Event received
final AbstractEvent event = bufferOrEvent.getEvent();
- handleEvent(event);
+ if (event.getClass() != EndOfPartitionEvent.class) {
+ throw new IOException("Unexpected event: " + event);
+ }
}
}
else {
@@ -185,15 +189,13 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
}
}
}
-
- @Override
+
public void setReporter(AccumulatorRegistry.Reporter reporter) {
for (RecordDeserializer<?> deserializer : recordDeserializers) {
deserializer.setReporter(reporter);
}
}
-
- @Override
+
public void cleanup() throws IOException {
// clear the buffers first. this part should not ever fail
for (RecordDeserializer<?> deserializer : recordDeserializers) {
http://git-wip-us.apache.org/repos/asf/flink/blob/b18e410b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
index 7dffa71..6322cc8 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
@@ -22,8 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
import org.apache.flink.runtime.event.AbstractEvent;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.network.api.reader.AbstractReader;
-import org.apache.flink.runtime.io.network.api.reader.ReaderBase;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult;
import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer;
@@ -41,8 +40,6 @@ import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordS
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Arrays;
@@ -58,10 +55,7 @@ import java.util.Collection;
* @param <IN1> The type of the records that arrive on the first input
* @param <IN2> The type of the records that arrive on the second input
*/
-public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements ReaderBase, StreamingReader {
-
- @SuppressWarnings("unused")
- private static final Logger LOG = LoggerFactory.getLogger(StreamTwoInputProcessor.class);
+public class StreamTwoInputProcessor<IN1, IN2> {
private final RecordDeserializer<DeserializationDelegate<StreamElement>>[] recordDeserializers;
@@ -97,7 +91,7 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
IOManager ioManager,
boolean enableWatermarkMultiplexing) throws IOException {
- super(InputGateUtil.createInputGate(inputGates1, inputGates2));
+ final InputGate inputGate = InputGateUtil.createInputGate(inputGates1, inputGates2);
if (checkpointMode == CheckpointingMode.EXACTLY_ONCE) {
this.barrierHandler = new BarrierBuffer(inputGate, ioManager);
@@ -157,8 +151,7 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
Arrays.fill(watermarks2, Long.MIN_VALUE);
lastEmittedWatermark2 = Long.MIN_VALUE;
}
-
- @SuppressWarnings("unchecked")
+
public boolean processInput(TwoInputStreamOperator<IN1, IN2, ?> streamOperator) throws Exception {
if (isFinished) {
return false;
@@ -216,7 +209,9 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
} else {
// Event received
final AbstractEvent event = bufferOrEvent.getEvent();
- handleEvent(event);
+ if (event.getClass() != EndOfPartitionEvent.class) {
+ throw new IOException("Unexpected event: " + event);
+ }
}
}
else {
@@ -259,15 +254,13 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
}
}
}
-
- @Override
+
public void setReporter(AccumulatorRegistry.Reporter reporter) {
for (RecordDeserializer<?> deserializer : recordDeserializers) {
deserializer.setReporter(reporter);
}
}
-
- @Override
+
public void cleanup() throws IOException {
// clear the buffers first. this part should not ever fail
for (RecordDeserializer<?> deserializer : recordDeserializers) {
http://git-wip-us.apache.org/repos/asf/flink/blob/b18e410b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java
index 987a8fb..a6c6936 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java
@@ -32,13 +32,12 @@ import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.util.NoOpIntMap;
import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
import org.apache.flink.streaming.util.TestListResultSink;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
+
import org.junit.Test;
/**
@@ -147,7 +146,9 @@ public class PartitionerTest extends StreamingMultipleProgramsTestBase {
try {
env.execute();
- } catch (Exception e) {
+ }
+ catch (Exception e) {
+ e.printStackTrace();
fail(e.getMessage());
}
[10/15] flink git commit: [FLINK-1320] [core] Add an off-heap variant
of the managed memory
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegment.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegment.java
new file mode 100644
index 0000000..57817b9
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegment.java
@@ -0,0 +1,887 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory.benchmarks;
+
+import org.apache.flink.core.memory.MemoryUtils;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.nio.BufferOverflowException;
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+public final class PureHybridMemorySegment {
+
+ /** Constant that flags the byte order. Because this is a boolean constant,
+ * the JIT compiler can use this well to aggressively eliminate the non-applicable code paths */
+ private static final boolean LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+ /** The direct byte buffer that allocated the off-heap memory. This memory segment holds a reference
+ * to that buffer, so as long as this memory segment lives, the memory will not be released. */
+ private final ByteBuffer offHeapMemory;
+
+ /** The heap byte array object relative to which we access the memory. Is non-null if the
+ * memory is on the heap, is null, if the memory if off the heap. If we have this buffer, we
+ * must never void this reference, or the memory segment will point to undefined addresses
+ * outside the heap and may in out-of-order execution cases cause segmentation faults. */
+ private final byte[] heapMemory;
+
+ /** The address to the data, relative to the heap memory byte array. If the heap memory byte array
+ * is null, this becomes an absolute memory address outside the heap. */
+ private long address;
+
+ /** The address one byte after the last addressable byte.
+ * This is address + size while the segment is not disposed */
+ private final long addressLimit;
+
+ /** The size in bytes of the memory segment */
+ private final int size;
+
+ // -------------------------------------------------------------------------
+ // Constructors
+ // -------------------------------------------------------------------------
+
+ /**
+ * Creates a new memory segment that represents the memory backing the given direct byte buffer.
+ * Note that the given ByteBuffer must be direct {@link java.nio.ByteBuffer#allocateDirect(int)},
+ * otherwise this method with throw an IllegalArgumentException.
+ *
+ * @param buffer The byte buffer whose memory is represented by this memory segment.
+ * @throws IllegalArgumentException Thrown, if the given ByteBuffer is not direct.
+ */
+ public PureHybridMemorySegment(ByteBuffer buffer) {
+ if (buffer == null || !buffer.isDirect()) {
+ throw new IllegalArgumentException("Can't initialize from non-direct ByteBuffer.");
+ }
+
+ this.offHeapMemory = buffer;
+ this.heapMemory = null;
+ this.size = buffer.capacity();
+ this.address = getAddress(buffer);
+ this.addressLimit = this.address + size;
+
+ if (address >= Long.MAX_VALUE - Integer.MAX_VALUE) {
+ throw new RuntimeException("Segment initialized with too large address: " + address
+ + " ; Max allowed address is " + (Long.MAX_VALUE - Integer.MAX_VALUE - 1));
+ }
+ }
+
+ /**
+ * Creates a new memory segment that represents the memory of the byte array.
+ *
+ * @param buffer The byte array whose memory is represented by this memory segment.
+ */
+ public PureHybridMemorySegment(byte[] buffer) {
+ if (buffer == null) {
+ throw new NullPointerException("buffer");
+ }
+
+ this.offHeapMemory = null;
+ this.heapMemory = buffer;
+ this.address = BYTE_ARRAY_BASE_OFFSET;
+ this.addressLimit = BYTE_ARRAY_BASE_OFFSET + buffer.length;
+ this.size = buffer.length;
+ }
+
+ // -------------------------------------------------------------------------
+ // Memory Segment Specifics
+ // -------------------------------------------------------------------------
+
+ /**
+ * Gets the size of the memory segment, in bytes.
+ * @return The size of the memory segment.
+ */
+ public final int size() {
+ return size;
+ }
+
+ /**
+ * Checks whether the memory segment was freed.
+ * @return True, if the memory segment has been freed, false otherwise.
+ */
+ public final boolean isFreed() {
+ return this.address > this.addressLimit;
+ }
+
+ /**
+ * Frees this memory segment. After this operation has been called, no further operations are
+ * possible on the memory segment and will fail. The actual memory (heap or off-heap) will only
+ * be released after this memory segment object has become garbage collected.
+ */
+ public final void free() {
+ // this ensures we can place no more data and trigger
+ // the checks for the freed segment
+ address = addressLimit + 1;
+ }
+
+ /**
+ * Checks whether this memory segment is backed by off-heap memory.
+ * @return True, if the memory segment is backed by off-heap memory, false if it is backed
+ * by heap memory.
+ */
+ public final boolean isOffHeap() {
+ return heapMemory == null;
+ }
+
+ public byte[] getArray() {
+ if (heapMemory != null) {
+ return heapMemory;
+ } else {
+ throw new IllegalStateException("Memory segment does not represent heap memory");
+ }
+ }
+
+ /**
+ * Gets the buffer that owns the memory of this memory segment.
+ *
+ * @return The byte buffer that owns the memory of this memory segment.
+ */
+ public ByteBuffer getOffHeapBuffer() {
+ if (offHeapMemory != null) {
+ return offHeapMemory;
+ } else {
+ throw new IllegalStateException("Memory segment does not represent off heap memory");
+ }
+ }
+
+ public ByteBuffer wrap(int offset, int length) {
+ if (offset < 0 || offset > this.size || offset > this.size - length) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ if (heapMemory != null) {
+ return ByteBuffer.wrap(heapMemory, offset, length);
+ }
+ else {
+ ByteBuffer wrapper = offHeapMemory.duplicate();
+ wrapper.limit(offset + length);
+ wrapper.position(offset);
+ return wrapper;
+ }
+ }
+
+ /**
+ * Gets this memory segment as a pure heap memory segment.
+ *
+ * @return A heap memory segment variant of this memory segment.
+ * @throws UnsupportedOperationException Thrown, if this memory segment is not
+ * backed by heap memory.
+ */
+ public final PureHeapMemorySegment asHeapSegment() {
+ if (heapMemory != null) {
+ return new PureHeapMemorySegment(heapMemory);
+ } else {
+ throw new UnsupportedOperationException("Memory segment is not backed by heap memory");
+ }
+ }
+
+ /**
+ * Gets this memory segment as a pure off-heap memory segment.
+ *
+ * @return An off-heap memory segment variant of this memory segment.
+ * @throws UnsupportedOperationException Thrown, if this memory segment is not
+ * backed by off-heap memory.
+ */
+ public final PureOffHeapMemorySegment asOffHeapSegment() {
+ if (offHeapMemory != null) {
+ return new PureOffHeapMemorySegment(offHeapMemory);
+ } else {
+ throw new UnsupportedOperationException("Memory segment is not backed by off-heap memory");
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Random Access get() and put() methods
+ // ------------------------------------------------------------------------
+
+ @SuppressWarnings("restriction")
+ public final byte get(int index) {
+ final long pos = address + index;
+ if (index >= 0 && pos < addressLimit) {
+ return UNSAFE.getByte(heapMemory, pos);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void put(int index, byte b) {
+ final long pos = address + index;
+ if (index >= 0 && pos < addressLimit) {
+ UNSAFE.putByte(heapMemory, pos, b);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final void get(int index, byte[] dst) {
+ get(index, dst, 0, dst.length);
+ }
+
+ public final void put(int index, byte[] src) {
+ put(index, src, 0, src.length);
+ }
+
+ @SuppressWarnings("restriction")
+ public final void get(int index, byte[] dst, int offset, int length) {
+ // check the byte array offset and length
+ if ((offset | length | (offset + length) | (dst.length - (offset + length))) < 0) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ long pos = address + index;
+
+ if (index >= 0 && pos <= addressLimit - length) {
+ long arrayAddress = BYTE_ARRAY_BASE_OFFSET + offset;
+
+ // the copy must proceed in batches not too large, because the JVM may
+ // poll for points that are safe for GC (moving the array and changing its address)
+ while (length > 0) {
+ long toCopy = Math.min(length, COPY_PER_BATCH);
+ UNSAFE.copyMemory(heapMemory, pos, dst, arrayAddress, toCopy);
+ length -= toCopy;
+ pos += toCopy;
+ arrayAddress += toCopy;
+ }
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void put(int index, byte[] src, int offset, int length) {
+ // check the byte array offset and length
+ if ((offset | length | (offset + length) | (src.length - (offset + length))) < 0) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ long pos = address + index;
+
+ if (index >= 0 && pos <= addressLimit - length) {
+ long arrayAddress = BYTE_ARRAY_BASE_OFFSET + offset;
+ while (length > 0) {
+ long toCopy = Math.min(length, COPY_PER_BATCH);
+ UNSAFE.copyMemory(src, arrayAddress, heapMemory, pos, toCopy);
+ length -= toCopy;
+ pos += toCopy;
+ arrayAddress += toCopy;
+ }
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final boolean getBoolean(int index) {
+ return get(index) != 0;
+ }
+
+ public final void putBoolean(int index, boolean value) {
+ put(index, (byte) (value ? 1 : 0));
+ }
+
+ @SuppressWarnings("restriction")
+ public final char getChar(int index) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 2) {
+ return UNSAFE.getChar(heapMemory, pos);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final char getCharLittleEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return getChar(index);
+ } else {
+ return Character.reverseBytes(getChar(index));
+ }
+ }
+
+ public final char getCharBigEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return Character.reverseBytes(getChar(index));
+ } else {
+ return getChar(index);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void putChar(int index, char value) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 2) {
+ UNSAFE.putChar(heapMemory, pos, value);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final void putCharLittleEndian(int index, char value) {
+ if (LITTLE_ENDIAN) {
+ putChar(index, value);
+ } else {
+ putChar(index, Character.reverseBytes(value));
+ }
+ }
+
+ public final void putCharBigEndian(int index, char value) {
+ if (LITTLE_ENDIAN) {
+ putChar(index, Character.reverseBytes(value));
+ } else {
+ putChar(index, value);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final short getShort(int index) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 2) {
+ return UNSAFE.getShort(heapMemory, pos);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final short getShortLittleEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return getShort(index);
+ } else {
+ return Short.reverseBytes(getShort(index));
+ }
+ }
+
+ public final short getShortBigEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return Short.reverseBytes(getShort(index));
+ } else {
+ return getShort(index);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void putShort(int index, short value) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 2) {
+ UNSAFE.putShort(heapMemory, pos, value);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final void putShortLittleEndian(int index, short value) {
+ if (LITTLE_ENDIAN) {
+ putShort(index, value);
+ } else {
+ putShort(index, Short.reverseBytes(value));
+ }
+ }
+
+ public final void putShortBigEndian(int index, short value) {
+ if (LITTLE_ENDIAN) {
+ putShort(index, Short.reverseBytes(value));
+ } else {
+ putShort(index, value);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final int getInt(int index) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 4) {
+ return UNSAFE.getInt(heapMemory, pos);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final int getIntLittleEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return getInt(index);
+ } else {
+ return Integer.reverseBytes(getInt(index));
+ }
+ }
+
+ public final int getIntBigEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return Integer.reverseBytes(getInt(index));
+ } else {
+ return getInt(index);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void putInt(int index, int value) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 4) {
+ UNSAFE.putInt(heapMemory, pos, value);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final void putIntLittleEndian(int index, int value) {
+ if (LITTLE_ENDIAN) {
+ putInt(index, value);
+ } else {
+ putInt(index, Integer.reverseBytes(value));
+ }
+ }
+
+ public final void putIntBigEndian(int index, int value) {
+ if (LITTLE_ENDIAN) {
+ putInt(index, Integer.reverseBytes(value));
+ } else {
+ putInt(index, value);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final long getLong(int index) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 8) {
+ return UNSAFE.getLong(heapMemory, pos);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final long getLongLittleEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return getLong(index);
+ } else {
+ return Long.reverseBytes(getLong(index));
+ }
+ }
+
+ public final long getLongBigEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return Long.reverseBytes(getLong(index));
+ } else {
+ return getLong(index);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void putLong(int index, long value) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 8) {
+ UNSAFE.putLong(heapMemory, pos, value);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final void putLongLittleEndian(int index, long value) {
+ if (LITTLE_ENDIAN) {
+ putLong(index, value);
+ } else {
+ putLong(index, Long.reverseBytes(value));
+ }
+ }
+
+ public final void putLongBigEndian(int index, long value) {
+ if (LITTLE_ENDIAN) {
+ putLong(index, Long.reverseBytes(value));
+ } else {
+ putLong(index, value);
+ }
+ }
+
+ public final float getFloat(int index) {
+ return Float.intBitsToFloat(getInt(index));
+ }
+
+ public final float getFloatLittleEndian(int index) {
+ return Float.intBitsToFloat(getIntLittleEndian(index));
+ }
+
+ public final float getFloatBigEndian(int index) {
+ return Float.intBitsToFloat(getIntBigEndian(index));
+ }
+
+ public final void putFloat(int index, float value) {
+ putInt(index, Float.floatToRawIntBits(value));
+ }
+
+ public final void putFloatLittleEndian(int index, float value) {
+ putIntLittleEndian(index, Float.floatToRawIntBits(value));
+ }
+
+ public final void putFloatBigEndian(int index, float value) {
+ putIntBigEndian(index, Float.floatToRawIntBits(value));
+ }
+
+ public final double getDouble(int index) {
+ return Double.longBitsToDouble(getLong(index));
+ }
+
+ public final double getDoubleLittleEndian(int index) {
+ return Double.longBitsToDouble(getLongLittleEndian(index));
+ }
+
+ public final double getDoubleBigEndian(int index) {
+ return Double.longBitsToDouble(getLongBigEndian(index));
+ }
+
+ public final void putDouble(int index, double value) {
+ putLong(index, Double.doubleToRawLongBits(value));
+ }
+
+ public final void putDoubleLittleEndian(int index, double value) {
+ putLongLittleEndian(index, Double.doubleToRawLongBits(value));
+ }
+
+ public final void putDoubleBigEndian(int index, double value) {
+ putLongBigEndian(index, Double.doubleToRawLongBits(value));
+ }
+
+ // -------------------------------------------------------------------------
+ // Bulk Read and Write Methods
+ // -------------------------------------------------------------------------
+
+ public final void get(DataOutput out, int offset, int length) throws IOException {
+ if (heapMemory != null) {
+ out.write(heapMemory, offset, length);
+ }
+ else {
+ while (length >= 8) {
+ out.writeLong(getLongBigEndian(offset));
+ offset += 8;
+ length -= 8;
+ }
+
+ while (length > 0) {
+ out.writeByte(get(offset));
+ offset++;
+ length--;
+ }
+ }
+ }
+
+ public final void put(DataInput in, int offset, int length) throws IOException {
+ if (heapMemory != null) {
+ in.readFully(heapMemory, offset, length);
+ }
+ else {
+ while (length >= 8) {
+ putLongBigEndian(offset, in.readLong());
+ offset += 8;
+ length -= 8;
+ }
+ while(length > 0) {
+ put(offset, in.readByte());
+ offset++;
+ length--;
+ }
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void get(int offset, ByteBuffer target, int numBytes) {
+ if (heapMemory != null) {
+ // ByteBuffer performs the boundary checks
+ target.put(heapMemory, offset, numBytes);
+ }
+ else {
+ // check the byte array offset and length
+ if ((offset | numBytes | (offset + numBytes) | (size - (offset + numBytes))) < 0) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ final int targetOffset = target.position();
+ final int remaining = target.remaining();
+
+ if (remaining < numBytes) {
+ throw new BufferOverflowException();
+ }
+
+ if (target.isDirect()) {
+ // copy to the target memory directly
+ final long targetPointer = getAddress(target) + targetOffset;
+ final long sourcePointer = address + offset;
+
+ if (sourcePointer <= addressLimit - numBytes) {
+ UNSAFE.copyMemory(sourcePointer, targetPointer, numBytes);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+ else if (target.hasArray()) {
+ // move directly into the byte array
+ get(offset, target.array(), targetOffset + target.arrayOffset(), numBytes);
+
+ // this must be after the get() call to ensue that the byte buffer is not
+ // modified in case the call fails
+ target.position(targetOffset + numBytes);
+ }
+ else {
+ // neither heap buffer nor direct buffer
+ while (target.hasRemaining()) {
+ target.put(get(offset++));
+ }
+ }
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void put(int offset, ByteBuffer source, int numBytes) {
+ if (heapMemory != null) {
+ source.get(heapMemory, offset, numBytes);
+ }
+ else {
+ // check the byte array offset and length
+ if ((offset | numBytes | (offset + numBytes) | (size - (offset + numBytes))) < 0) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ final int sourceOffset = source.position();
+ final int remaining = source.remaining();
+
+ if (remaining < numBytes) {
+ throw new BufferUnderflowException();
+ }
+
+ if (source.isDirect()) {
+ // copy to the target memory directly
+ final long sourcePointer = getAddress(source) + sourceOffset;
+ final long targetPointer = address + offset;
+
+ if (sourcePointer <= addressLimit - numBytes) {
+ UNSAFE.copyMemory(sourcePointer, targetPointer, numBytes);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+ else if (source.hasArray()) {
+ // move directly into the byte array
+ put(offset, source.array(), sourceOffset + source.arrayOffset(), numBytes);
+
+ // this must be after the get() call to ensue that the byte buffer is not
+ // modified in case the call fails
+ source.position(sourceOffset + numBytes);
+ }
+ else {
+ // neither heap buffer nor direct buffer
+ while (source.hasRemaining()) {
+ put(offset++, source.get());
+ }
+ }
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void copyTo(int offset, PureHybridMemorySegment target, int targetOffset, int numBytes) {
+ final byte[] thisHeapRef = this.heapMemory;
+ final byte[] otherHeapRef = target.heapMemory;
+ final long thisPointer = this.address + offset;
+ final long otherPointer = target.address + targetOffset;
+
+ if (numBytes >= 0 & thisPointer <= this.addressLimit - numBytes & otherPointer <= target.addressLimit - numBytes) {
+ UNSAFE.copyMemory(thisHeapRef, thisPointer, otherHeapRef, otherPointer, numBytes);
+ }
+ else if (address > addressLimit | target.address > target.addressLimit) {
+ throw new IllegalStateException("segment has been freed.");
+ }
+ else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public int compare(PureHybridMemorySegment seg2, int offset1, int offset2, int len) {
+ while (len >= 8) {
+ long l1 = this.getLongBigEndian(offset1);
+ long l2 = seg2.getLongBigEndian(offset2);
+
+ if (l1 != l2) {
+ return (l1 < l2) ^ (l1 < 0) ^ (l2 < 0) ? -1 : 1;
+ }
+
+ offset1 += 8;
+ offset2 += 8;
+ len -= 8;
+ }
+ while (len > 0) {
+ int b1 = this.get(offset1) & 0xff;
+ int b2 = seg2.get(offset2) & 0xff;
+ int cmp = b1 - b2;
+ if (cmp != 0) {
+ return cmp;
+ }
+ offset1++;
+ offset2++;
+ len--;
+ }
+ return 0;
+ }
+
+ public void swapBytes(byte[] tempBuffer, PureHybridMemorySegment seg2, int offset1, int offset2, int len) {
+ if (len < 32) {
+ // fast path for short copies
+ while (len >= 8) {
+ long tmp = this.getLong(offset1);
+ this.putLong(offset1, seg2.getLong(offset2));
+ seg2.putLong(offset2, tmp);
+ offset1 += 8;
+ offset2 += 8;
+ len -= 8;
+ }
+ while (len > 0) {
+ byte tmp = this.get(offset1);
+ this.put(offset1, seg2.get(offset2));
+ seg2.put(offset2, tmp);
+ offset1++;
+ offset2++;
+ len--;
+ }
+ }
+ else if ( (offset1 | offset2 | len | (offset1 + len) | (offset2 + len) |
+ (this.size - (offset1 + len)) | (seg2.size() - (offset2 + len))) < 0 || len > tempBuffer.length)
+ {
+ throw new IndexOutOfBoundsException();
+ }
+ else {
+ final long thisPos = this.address + offset1;
+ final long otherPos = seg2.address + offset2;
+
+ if (thisPos <= this.addressLimit - len && otherPos <= seg2.addressLimit - len) {
+ final long arrayAddress = BYTE_ARRAY_BASE_OFFSET;
+
+ // this -> temp buffer
+ UNSAFE.copyMemory(this.heapMemory, thisPos, tempBuffer, arrayAddress, len);
+
+ // other -> this
+ UNSAFE.copyMemory(seg2.heapMemory, otherPos, this.heapMemory, thisPos, len);
+
+ // temp buffer -> other
+ UNSAFE.copyMemory(tempBuffer, arrayAddress, seg2.heapMemory, otherPos, len);
+ }
+ else if (this.address <= 0 || seg2.address <= 0) {
+ throw new IllegalStateException("Memory segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Utilities for native memory accesses and checks
+ // --------------------------------------------------------------------------------------------
+
+ @SuppressWarnings("restriction")
+ private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+
+ @SuppressWarnings("restriction")
+ private static final long BYTE_ARRAY_BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+
+ private static final long COPY_PER_BATCH = 1024 * 1024;
+
+ private static final Field ADDRESS_FIELD;
+
+ static {
+ try {
+ ADDRESS_FIELD = java.nio.Buffer.class.getDeclaredField("address");
+ ADDRESS_FIELD.setAccessible(true);
+ }
+ catch (Throwable t) {
+ throw new RuntimeException("Cannot initialize DirectMemorySegment - direct memory not supported by the JVM.");
+ }
+ }
+
+ private static long getAddress(ByteBuffer buf) {
+ try {
+ return (Long) ADDRESS_FIELD.get(buf);
+ }
+ catch (Throwable t) {
+ throw new RuntimeException("Could not access direct byte buffer address.", t);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegmentOutView.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegmentOutView.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegmentOutView.java
new file mode 100644
index 0000000..cda48e1
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegmentOutView.java
@@ -0,0 +1,359 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory.benchmarks;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.util.List;
+
+public final class PureHybridMemorySegmentOutView implements DataOutputView {
+
+ private PureHybridMemorySegment currentSegment; // the current memory segment to write to
+
+ private int positionInSegment; // the offset in the current segment
+
+ private final int segmentSize; // the size of the memory segments
+
+ private final List<PureHybridMemorySegment> memorySource;
+
+ private final List<PureHybridMemorySegment> fullSegments;
+
+
+ private byte[] utfBuffer; // the reusable array for UTF encodings
+
+
+ public PureHybridMemorySegmentOutView(List<PureHybridMemorySegment> emptySegments,
+ List<PureHybridMemorySegment> fullSegmentTarget, int segmentSize) {
+ this.segmentSize = segmentSize;
+ this.currentSegment = emptySegments.remove(emptySegments.size() - 1);
+
+ this.memorySource = emptySegments;
+ this.fullSegments = fullSegmentTarget;
+ this.fullSegments.add(getCurrentSegment());
+ }
+
+
+ public void reset() {
+ if (this.fullSegments.size() != 0) {
+ throw new IllegalStateException("The target list still contains memory segments.");
+ }
+
+ clear();
+ try {
+ advance();
+ }
+ catch (IOException ioex) {
+ throw new RuntimeException("Error getting first segment for record collector.", ioex);
+ }
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Page Management
+ // --------------------------------------------------------------------------------------------
+
+ public PureHybridMemorySegment nextSegment(PureHybridMemorySegment current, int positionInCurrent) throws EOFException {
+ int size = this.memorySource.size();
+ if (size > 0) {
+ final PureHybridMemorySegment next = this.memorySource.remove(size - 1);
+ this.fullSegments.add(next);
+ return next;
+ } else {
+ throw new EOFException();
+ }
+ }
+
+ public PureHybridMemorySegment getCurrentSegment() {
+ return this.currentSegment;
+ }
+
+ public int getCurrentPositionInSegment() {
+ return this.positionInSegment;
+ }
+
+ public int getSegmentSize() {
+ return this.segmentSize;
+ }
+
+ protected void advance() throws IOException {
+ this.currentSegment = nextSegment(this.currentSegment, this.positionInSegment);
+ this.positionInSegment = 0;
+ }
+
+ protected void seekOutput(PureHybridMemorySegment seg, int position) {
+ this.currentSegment = seg;
+ this.positionInSegment = position;
+ }
+
+ protected void clear() {
+ this.currentSegment = null;
+ this.positionInSegment = 0;
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Data Output Specific methods
+ // --------------------------------------------------------------------------------------------
+
+ @Override
+ public void write(int b) throws IOException {
+ writeByte(b);
+ }
+
+ @Override
+ public void write(byte[] b) throws IOException {
+ write(b, 0, b.length);
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ int remaining = this.segmentSize - this.positionInSegment;
+ if (remaining >= len) {
+ this.currentSegment.put(this.positionInSegment, b, off, len);
+ this.positionInSegment += len;
+ }
+ else {
+ if (remaining == 0) {
+ advance();
+ remaining = this.segmentSize - this.positionInSegment;
+ }
+ while (true) {
+ int toPut = Math.min(remaining, len);
+ this.currentSegment.put(this.positionInSegment, b, off, toPut);
+ off += toPut;
+ len -= toPut;
+
+ if (len > 0) {
+ this.positionInSegment = this.segmentSize;
+ advance();
+ remaining = this.segmentSize - this.positionInSegment;
+ }
+ else {
+ this.positionInSegment += toPut;
+ break;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void writeBoolean(boolean v) throws IOException {
+ writeByte(v ? 1 : 0);
+ }
+
+ @Override
+ public void writeByte(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize) {
+ this.currentSegment.put(this.positionInSegment++, (byte) v);
+ }
+ else {
+ advance();
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeShort(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 1) {
+ this.currentSegment.putShortBigEndian(this.positionInSegment, (short) v);
+ this.positionInSegment += 2;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeShort(v);
+ }
+ else {
+ writeByte(v >> 8);
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeChar(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 1) {
+ this.currentSegment.putCharBigEndian(this.positionInSegment, (char) v);
+ this.positionInSegment += 2;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeChar(v);
+ }
+ else {
+ writeByte(v >> 8);
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeInt(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 3) {
+ this.currentSegment.putIntBigEndian(this.positionInSegment, v);
+ this.positionInSegment += 4;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeInt(v);
+ }
+ else {
+ writeByte(v >> 24);
+ writeByte(v >> 16);
+ writeByte(v >> 8);
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeLong(long v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 7) {
+ this.currentSegment.putLongBigEndian(this.positionInSegment, v);
+ this.positionInSegment += 8;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeLong(v);
+ }
+ else {
+ writeByte((int) (v >> 56));
+ writeByte((int) (v >> 48));
+ writeByte((int) (v >> 40));
+ writeByte((int) (v >> 32));
+ writeByte((int) (v >> 24));
+ writeByte((int) (v >> 16));
+ writeByte((int) (v >> 8));
+ writeByte((int) v);
+ }
+ }
+
+ @Override
+ public void writeFloat(float v) throws IOException {
+ writeInt(Float.floatToRawIntBits(v));
+ }
+
+ @Override
+ public void writeDouble(double v) throws IOException {
+ writeLong(Double.doubleToRawLongBits(v));
+ }
+
+ @Override
+ public void writeBytes(String s) throws IOException {
+ for (int i = 0; i < s.length(); i++) {
+ writeByte(s.charAt(i));
+ }
+ }
+
+ @Override
+ public void writeChars(String s) throws IOException {
+ for (int i = 0; i < s.length(); i++) {
+ writeChar(s.charAt(i));
+ }
+ }
+
+ @Override
+ public void writeUTF(String str) throws IOException {
+ int strlen = str.length();
+ int utflen = 0;
+ int c, count = 0;
+
+ /* use charAt instead of copying String to char array */
+ for (int i = 0; i < strlen; i++) {
+ c = str.charAt(i);
+ if ((c >= 0x0001) && (c <= 0x007F)) {
+ utflen++;
+ } else if (c > 0x07FF) {
+ utflen += 3;
+ } else {
+ utflen += 2;
+ }
+ }
+
+ if (utflen > 65535) {
+ throw new UTFDataFormatException("encoded string too long: " + utflen + " memory");
+ }
+
+ if (this.utfBuffer == null || this.utfBuffer.length < utflen + 2) {
+ this.utfBuffer = new byte[utflen + 2];
+ }
+ final byte[] bytearr = this.utfBuffer;
+
+ bytearr[count++] = (byte) ((utflen >>> 8) & 0xFF);
+ bytearr[count++] = (byte) (utflen & 0xFF);
+
+ int i = 0;
+ for (i = 0; i < strlen; i++) {
+ c = str.charAt(i);
+ if (!((c >= 0x0001) && (c <= 0x007F))) {
+ break;
+ }
+ bytearr[count++] = (byte) c;
+ }
+
+ for (; i < strlen; i++) {
+ c = str.charAt(i);
+ if ((c >= 0x0001) && (c <= 0x007F)) {
+ bytearr[count++] = (byte) c;
+
+ } else if (c > 0x07FF) {
+ bytearr[count++] = (byte) (0xE0 | ((c >> 12) & 0x0F));
+ bytearr[count++] = (byte) (0x80 | ((c >> 6) & 0x3F));
+ bytearr[count++] = (byte) (0x80 | (c & 0x3F));
+ } else {
+ bytearr[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F));
+ bytearr[count++] = (byte) (0x80 | (c & 0x3F));
+ }
+ }
+
+ write(bytearr, 0, utflen + 2);
+ }
+
+ @Override
+ public void skipBytesToWrite(int numBytes) throws IOException {
+ while (numBytes > 0) {
+ final int remaining = this.segmentSize - this.positionInSegment;
+ if (numBytes <= remaining) {
+ this.positionInSegment += numBytes;
+ return;
+ }
+ this.positionInSegment = this.segmentSize;
+ advance();
+ numBytes -= remaining;
+ }
+ }
+
+ @Override
+ public void write(DataInputView source, int numBytes) throws IOException {
+ while (numBytes > 0) {
+ final int remaining = this.segmentSize - this.positionInSegment;
+ if (numBytes <= remaining) {
+ this.currentSegment.put(source, this.positionInSegment, numBytes);
+ this.positionInSegment += numBytes;
+ return;
+ }
+
+ if (remaining > 0) {
+ this.currentSegment.put(source, this.positionInSegment, remaining);
+ this.positionInSegment = this.segmentSize;
+ numBytes -= remaining;
+ }
+
+ advance();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegment.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegment.java
new file mode 100644
index 0000000..1280242
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegment.java
@@ -0,0 +1,790 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory.benchmarks;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryUtils;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.nio.BufferOverflowException;
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+public final class PureOffHeapMemorySegment {
+
+ /** Constant that flags the byte order. Because this is a boolean constant,
+ * the JIT compiler can use this well to aggressively eliminate the non-applicable code paths */
+ private static final boolean LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+ /** The direct byte buffer that allocated the memory */
+ private ByteBuffer buffer;
+
+ /** The address to the off-heap data */
+ private long address;
+
+ /** The address one byte after the last addressable byte.
+ * This is address + size while the segment is not disposed */
+ private final long addressLimit;
+
+ /** The size in bytes of the memory segment */
+ private final int size;
+
+ // -------------------------------------------------------------------------
+ // Constructors
+ // -------------------------------------------------------------------------
+
+ /**
+ * Creates a new memory segment that represents the memory backing the given direct byte buffer.
+ * Note that the given ByteBuffer must be direct {@link java.nio.ByteBuffer#allocateDirect(int)},
+ * otherwise this method with throw an IllegalArgumentException. data in the given byte array.
+ *
+ * @param buffer The byte buffer whose memory is represented by this memory segment.
+ * @throws IllegalArgumentException Thrown, if the given ByteBuffer is not direct.
+ */
+ public PureOffHeapMemorySegment(ByteBuffer buffer) {
+ if (buffer == null || !buffer.isDirect()) {
+ throw new IllegalArgumentException("Can't initialize from non-direct ByteBuffer.");
+ }
+
+ this.buffer = buffer;
+ this.size = buffer.capacity();
+ this.address = getAddress(buffer);
+ this.addressLimit = this.address + size;
+
+ if (address >= Long.MAX_VALUE - Integer.MAX_VALUE) {
+ throw new RuntimeException("Segment initialized with too large address: " + address);
+ }
+ }
+
+ // -------------------------------------------------------------------------
+ // Direct Memory Segment Specifics
+ // -------------------------------------------------------------------------
+
+ /**
+ * Gets the buffer that owns the memory of this memory segment.
+ *
+ * @return The byte buffer that owns the memory of this memory segment.
+ */
+ public ByteBuffer getBuffer() {
+ return this.buffer;
+ }
+
+ /**
+ * Gets the memory address of the memory backing this memory segment.
+ *
+ * @return The memory start address of the memory backing this memory segment.
+ */
+ public long getAddress() {
+ return address;
+ }
+
+ // -------------------------------------------------------------------------
+ // MemorySegment Accessors
+ // -------------------------------------------------------------------------
+
+ public final boolean isFreed() {
+ return this.address > this.addressLimit;
+ }
+
+ public final void free() {
+ // this ensures we can place no more data and trigger
+ // the checks for the freed segment
+ this.address = this.addressLimit + 1;
+ this.buffer = null;
+ }
+
+ public final int size() {
+ return this.size;
+ }
+
+ public ByteBuffer wrap(int offset, int length) {
+ if (offset < 0 || offset > this.size || offset > this.size - length) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ this.buffer.limit(offset + length);
+ this.buffer.position(offset);
+
+ return this.buffer;
+ }
+
+
+ // ------------------------------------------------------------------------
+ // Random Access get() and put() methods
+ // ------------------------------------------------------------------------
+
+ @SuppressWarnings("restriction")
+ public final byte get(int index) {
+
+ final long pos = address + index;
+ if (index >= 0 && pos < addressLimit) {
+ return UNSAFE.getByte(pos);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void put(int index, byte b) {
+
+ final long pos = address + index;
+ if (index >= 0 && pos < addressLimit) {
+ UNSAFE.putByte(pos, b);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final void get(int index, byte[] dst) {
+ get(index, dst, 0, dst.length);
+ }
+
+ public final void put(int index, byte[] src) {
+ put(index, src, 0, src.length);
+ }
+
+ @SuppressWarnings("restriction")
+ public final void get(int index, byte[] dst, int offset, int length) {
+
+ // check the byte array offset and length
+ if ((offset | length | (offset + length) | (dst.length - (offset + length))) < 0) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ long pos = address + index;
+
+ if (index >= 0 && pos <= addressLimit - length) {
+ long arrayAddress = BYTE_ARRAY_BASE_OFFSET + offset;
+
+ // the copy must proceed in batches not too large, because the JVM may
+ // poll for points that are safe for GC (moving the array and changing its address)
+ while (length > 0) {
+ long toCopy = (length > COPY_PER_BATCH) ? COPY_PER_BATCH : length;
+ UNSAFE.copyMemory(null, pos, dst, arrayAddress, toCopy);
+ length -= toCopy;
+ pos += toCopy;
+ arrayAddress += toCopy;
+ }
+ }
+ else if (address <= 0) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void put(int index, byte[] src, int offset, int length) {
+ // check the byte array offset and length
+ if ((offset | length | (offset + length) | (src.length - (offset + length))) < 0) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ long pos = address + index;
+
+ if (index >= 0 && pos <= addressLimit - length) {
+
+ long arrayAddress = BYTE_ARRAY_BASE_OFFSET + offset;
+ while (length > 0) {
+ long toCopy = (length > COPY_PER_BATCH) ? COPY_PER_BATCH : length;
+ UNSAFE.copyMemory(src, arrayAddress, null, pos, toCopy);
+ length -= toCopy;
+ pos += toCopy;
+ arrayAddress += toCopy;
+ }
+ }
+ else if (address <= 0) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final boolean getBoolean(int index) {
+ return get(index) != 0;
+ }
+
+ public final void putBoolean(int index, boolean value) {
+ put(index, (byte) (value ? 1 : 0));
+ }
+
+ @SuppressWarnings("restriction")
+ public final char getChar(int index) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 2) {
+ return UNSAFE.getChar(pos);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final char getCharLittleEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return getChar(index);
+ } else {
+ return Character.reverseBytes(getChar(index));
+ }
+ }
+
+ public final char getCharBigEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return Character.reverseBytes(getChar(index));
+ } else {
+ return getChar(index);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void putChar(int index, char value) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 2) {
+ UNSAFE.putChar(pos, value);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final void putCharLittleEndian(int index, char value) {
+ if (LITTLE_ENDIAN) {
+ putChar(index, value);
+ } else {
+ putChar(index, Character.reverseBytes(value));
+ }
+ }
+
+ public final void putCharBigEndian(int index, char value) {
+ if (LITTLE_ENDIAN) {
+ putChar(index, Character.reverseBytes(value));
+ } else {
+ putChar(index, value);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final short getShort(int index) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 2) {
+ return UNSAFE.getShort(pos);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final short getShortLittleEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return getShort(index);
+ } else {
+ return Short.reverseBytes(getShort(index));
+ }
+ }
+
+ public final short getShortBigEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return Short.reverseBytes(getShort(index));
+ } else {
+ return getShort(index);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void putShort(int index, short value) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 2) {
+ UNSAFE.putShort(pos, value);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final void putShortLittleEndian(int index, short value) {
+ if (LITTLE_ENDIAN) {
+ putShort(index, value);
+ } else {
+ putShort(index, Short.reverseBytes(value));
+ }
+ }
+
+ public final void putShortBigEndian(int index, short value) {
+ if (LITTLE_ENDIAN) {
+ putShort(index, Short.reverseBytes(value));
+ } else {
+ putShort(index, value);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final int getInt(int index) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 4) {
+ return UNSAFE.getInt(pos);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final int getIntLittleEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return getInt(index);
+ } else {
+ return Integer.reverseBytes(getInt(index));
+ }
+ }
+
+ public final int getIntBigEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return Integer.reverseBytes(getInt(index));
+ } else {
+ return getInt(index);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void putInt(int index, int value) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 4) {
+ UNSAFE.putInt(pos, value);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final void putIntLittleEndian(int index, int value) {
+ if (LITTLE_ENDIAN) {
+ putInt(index, value);
+ } else {
+ putInt(index, Integer.reverseBytes(value));
+ }
+ }
+
+ public final void putIntBigEndian(int index, int value) {
+ if (LITTLE_ENDIAN) {
+ putInt(index, Integer.reverseBytes(value));
+ } else {
+ putInt(index, value);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final long getLong(int index) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 8) {
+ return UNSAFE.getLong(pos);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final long getLongLittleEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return getLong(index);
+ } else {
+ return Long.reverseBytes(getLong(index));
+ }
+ }
+
+ public final long getLongBigEndian(int index) {
+ if (LITTLE_ENDIAN) {
+ return Long.reverseBytes(getLong(index));
+ } else {
+ return getLong(index);
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void putLong(int index, long value) {
+ final long pos = address + index;
+ if (index >= 0 && pos <= addressLimit - 8) {
+ UNSAFE.putLong(pos, value);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public final void putLongLittleEndian(int index, long value) {
+ if (LITTLE_ENDIAN) {
+ putLong(index, value);
+ } else {
+ putLong(index, Long.reverseBytes(value));
+ }
+ }
+
+ public final void putLongBigEndian(int index, long value) {
+ if (LITTLE_ENDIAN) {
+ putLong(index, Long.reverseBytes(value));
+ } else {
+ putLong(index, value);
+ }
+ }
+
+ public final float getFloat(int index) {
+ return Float.intBitsToFloat(getInt(index));
+ }
+
+ public final float getFloatLittleEndian(int index) {
+ return Float.intBitsToFloat(getIntLittleEndian(index));
+ }
+
+ public final float getFloatBigEndian(int index) {
+ return Float.intBitsToFloat(getIntBigEndian(index));
+ }
+
+ public final void putFloat(int index, float value) {
+ putInt(index, Float.floatToRawIntBits(value));
+ }
+
+ public final void putFloatLittleEndian(int index, float value) {
+ putIntLittleEndian(index, Float.floatToRawIntBits(value));
+ }
+
+ public final void putFloatBigEndian(int index, float value) {
+ putIntBigEndian(index, Float.floatToRawIntBits(value));
+ }
+
+ public final double getDouble(int index) {
+ return Double.longBitsToDouble(getLong(index));
+ }
+
+ public final double getDoubleLittleEndian(int index) {
+ return Double.longBitsToDouble(getLongLittleEndian(index));
+ }
+
+ public final double getDoubleBigEndian(int index) {
+ return Double.longBitsToDouble(getLongBigEndian(index));
+ }
+
+ public final void putDouble(int index, double value) {
+ putLong(index, Double.doubleToRawLongBits(value));
+ }
+
+ public final void putDoubleLittleEndian(int index, double value) {
+ putLongLittleEndian(index, Double.doubleToRawLongBits(value));
+ }
+
+ public final void putDoubleBigEndian(int index, double value) {
+ putLongBigEndian(index, Double.doubleToRawLongBits(value));
+ }
+
+ // -------------------------------------------------------------------------
+ // Bulk Read and Write Methods
+ // -------------------------------------------------------------------------
+
+ public final void get(DataOutput out, int offset, int length) throws IOException {
+ while (length >= 8) {
+ out.writeLong(getLongBigEndian(offset));
+ offset += 8;
+ length -= 8;
+ }
+
+ while(length > 0) {
+ out.writeByte(get(offset));
+ offset++;
+ length--;
+ }
+ }
+
+ public final void put(DataInput in, int offset, int length) throws IOException {
+ while (length >= 8) {
+ putLongBigEndian(offset, in.readLong());
+ offset += 8;
+ length -= 8;
+ }
+ while(length > 0) {
+ put(offset, in.readByte());
+ offset++;
+ length--;
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void get(int offset, ByteBuffer target, int numBytes) {
+
+ // check the byte array offset and length
+ if ((offset | numBytes | (offset + numBytes) | (size - (offset + numBytes))) < 0) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ final int targetOffset = target.position();
+ final int remaining = target.remaining();
+
+ if (remaining < numBytes) {
+ throw new BufferOverflowException();
+ }
+
+ if (target.isDirect()) {
+ // copy to the target memory directly
+ final long targetPointer = getAddress(target) + targetOffset;
+ final long sourcePointer = address + offset;
+
+ if (sourcePointer <= addressLimit - numBytes) {
+ UNSAFE.copyMemory(sourcePointer, targetPointer, numBytes);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+ else if (target.hasArray()) {
+ // move directly into the byte array
+ get(offset, target.array(), targetOffset + target.arrayOffset(), numBytes);
+
+ // this must be after the get() call to ensue that the byte buffer is not
+ // modified in case the call fails
+ target.position(targetOffset + numBytes);
+ }
+ else {
+ // neither heap buffer nor direct buffer
+ while (target.hasRemaining()) {
+ target.put(get(offset++));
+ }
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void put(int offset, ByteBuffer source, int numBytes) {
+
+ // check the byte array offset and length
+ if ((offset | numBytes | (offset + numBytes) | (size - (offset + numBytes))) < 0) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ final int sourceOffset = source.position();
+ final int remaining = source.remaining();
+
+ if (remaining < numBytes) {
+ throw new BufferUnderflowException();
+ }
+
+ if (source.isDirect()) {
+ // copy to the target memory directly
+ final long sourcePointer = getAddress(source) + sourceOffset;
+ final long targetPointer = address + offset;
+
+ if (sourcePointer <= addressLimit - numBytes) {
+ UNSAFE.copyMemory(sourcePointer, targetPointer, numBytes);
+ }
+ else if (address > addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+ else if (source.hasArray()) {
+ // move directly into the byte array
+ put(offset, source.array(), sourceOffset + source.arrayOffset(), numBytes);
+
+ // this must be after the get() call to ensue that the byte buffer is not
+ // modified in case the call fails
+ source.position(sourceOffset + numBytes);
+ }
+ else {
+ // neither heap buffer nor direct buffer
+ while (source.hasRemaining()) {
+ put(offset++, source.get());
+ }
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ public final void copyTo(int offset, PureOffHeapMemorySegment target, int targetOffset, int numBytes) {
+ final long thisPointer = address + offset;
+ final long otherPointer = target.address + targetOffset;
+
+ if (numBytes >= 0 && thisPointer <= addressLimit - numBytes && otherPointer <= target.addressLimit - numBytes) {
+ UNSAFE.copyMemory(thisPointer, otherPointer, numBytes);
+ }
+ else if (address > addressLimit || target.address > target.addressLimit) {
+ throw new IllegalStateException("This segment has been freed.");
+ }
+ else {
+ throw new IndexOutOfBoundsException();
+ }
+ }
+
+ public int compare(MemorySegment seg2, int offset1, int offset2, int len) {
+ while (len >= 8) {
+ long l1 = this.getLongBigEndian(offset1);
+ long l2 = seg2.getLongBigEndian(offset2);
+
+ if (l1 != l2) {
+ return (l1 < l2) ^ (l1 < 0) ^ (l2 < 0) ? -1 : 1;
+ }
+
+ offset1 += 8;
+ offset2 += 8;
+ len -= 8;
+ }
+ while (len > 0) {
+ int b1 = this.get(offset1) & 0xff;
+ int b2 = seg2.get(offset2) & 0xff;
+ int cmp = b1 - b2;
+ if (cmp != 0) {
+ return cmp;
+ }
+ offset1++;
+ offset2++;
+ len--;
+ }
+ return 0;
+ }
+
+ public void swapBytes(byte[] tempBuffer, PureOffHeapMemorySegment seg2, int offset1, int offset2, int len) {
+ if (len < 32) {
+ // fast path for short copies
+ while (len >= 8) {
+ long tmp = this.getLong(offset1);
+ this.putLong(offset1, seg2.getLong(offset2));
+ seg2.putLong(offset2, tmp);
+ offset1 += 8;
+ offset2 += 8;
+ len -= 8;
+ }
+ while (len > 0) {
+ byte tmp = this.get(offset1);
+ this.put(offset1, seg2.get(offset2));
+ seg2.put(offset2, tmp);
+ offset1++;
+ offset2++;
+ len--;
+ }
+ }
+ else if ( (offset1 | offset2 | len | (offset1 + len) | (offset2 + len) |
+ (this.size - (offset1 + len)) | (seg2.size() - (offset2 + len))) < 0 || len > tempBuffer.length)
+ {
+ throw new IndexOutOfBoundsException();
+ }
+ else {
+ final long thisPos = this.address + offset1;
+ final long otherPos = seg2.address + offset2;
+
+ if (thisPos <= this.addressLimit - len && otherPos <= seg2.addressLimit - len) {
+ final long arrayAddress = BYTE_ARRAY_BASE_OFFSET;
+
+ // this -> temp buffer
+ UNSAFE.copyMemory(null, thisPos, tempBuffer, arrayAddress, len);
+
+ // other -> this
+ UNSAFE.copyMemory(null, otherPos, null, thisPos, len);
+
+ // temp buffer -> other
+ UNSAFE.copyMemory(tempBuffer, arrayAddress, null, otherPos, len);
+ }
+ else if (this.address <= 0 || seg2.address <= 0) {
+ throw new IllegalStateException("Memory segment has been freed.");
+ }
+ else {
+ // index is in fact invalid
+ throw new IndexOutOfBoundsException();
+ }
+ }
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Utilities for native memory accesses and checks
+ // --------------------------------------------------------------------------------------------
+
+ @SuppressWarnings("restriction")
+ private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+
+ @SuppressWarnings("restriction")
+ private static final long BYTE_ARRAY_BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+
+ private static final long COPY_PER_BATCH = 1024 * 1024;
+
+ private static final Field ADDRESS_FIELD;
+
+ static {
+ try {
+ ADDRESS_FIELD = java.nio.Buffer.class.getDeclaredField("address");
+ ADDRESS_FIELD.setAccessible(true);
+ }
+ catch (Throwable t) {
+ throw new RuntimeException("Cannot initialize DirectMemorySegment - direct memory not supported by the JVM.");
+ }
+ }
+
+ private static long getAddress(ByteBuffer buf) {
+ try {
+ return (Long) ADDRESS_FIELD.get(buf);
+ }
+ catch (Throwable t) {
+ throw new RuntimeException("Could not access direct byte buffer address.", t);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegmentOutView.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegmentOutView.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegmentOutView.java
new file mode 100644
index 0000000..d898dee
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegmentOutView.java
@@ -0,0 +1,359 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory.benchmarks;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.util.List;
+
+public final class PureOffHeapMemorySegmentOutView implements DataOutputView {
+
+ private PureOffHeapMemorySegment currentSegment; // the current memory segment to write to
+
+ private int positionInSegment; // the offset in the current segment
+
+ private final int segmentSize; // the size of the memory segments
+
+ private final List<PureOffHeapMemorySegment> memorySource;
+
+ private final List<PureOffHeapMemorySegment> fullSegments;
+
+
+ private byte[] utfBuffer; // the reusable array for UTF encodings
+
+
+ public PureOffHeapMemorySegmentOutView(List<PureOffHeapMemorySegment> emptySegments,
+ List<PureOffHeapMemorySegment> fullSegmentTarget, int segmentSize) {
+ this.segmentSize = segmentSize;
+ this.currentSegment = emptySegments.remove(emptySegments.size() - 1);
+
+ this.memorySource = emptySegments;
+ this.fullSegments = fullSegmentTarget;
+ this.fullSegments.add(getCurrentSegment());
+ }
+
+
+ public void reset() {
+ if (this.fullSegments.size() != 0) {
+ throw new IllegalStateException("The target list still contains memory segments.");
+ }
+
+ clear();
+ try {
+ advance();
+ }
+ catch (IOException ioex) {
+ throw new RuntimeException("Error getting first segment for record collector.", ioex);
+ }
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Page Management
+ // --------------------------------------------------------------------------------------------
+
+ public PureOffHeapMemorySegment nextSegment(PureOffHeapMemorySegment current, int positionInCurrent) throws EOFException {
+ int size = this.memorySource.size();
+ if (size > 0) {
+ final PureOffHeapMemorySegment next = this.memorySource.remove(size - 1);
+ this.fullSegments.add(next);
+ return next;
+ } else {
+ throw new EOFException();
+ }
+ }
+
+ public PureOffHeapMemorySegment getCurrentSegment() {
+ return this.currentSegment;
+ }
+
+ public int getCurrentPositionInSegment() {
+ return this.positionInSegment;
+ }
+
+ public int getSegmentSize() {
+ return this.segmentSize;
+ }
+
+ protected void advance() throws IOException {
+ this.currentSegment = nextSegment(this.currentSegment, this.positionInSegment);
+ this.positionInSegment = 0;
+ }
+
+ protected void seekOutput(PureOffHeapMemorySegment seg, int position) {
+ this.currentSegment = seg;
+ this.positionInSegment = position;
+ }
+
+ protected void clear() {
+ this.currentSegment = null;
+ this.positionInSegment = 0;
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Data Output Specific methods
+ // --------------------------------------------------------------------------------------------
+
+ @Override
+ public void write(int b) throws IOException {
+ writeByte(b);
+ }
+
+ @Override
+ public void write(byte[] b) throws IOException {
+ write(b, 0, b.length);
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ int remaining = this.segmentSize - this.positionInSegment;
+ if (remaining >= len) {
+ this.currentSegment.put(this.positionInSegment, b, off, len);
+ this.positionInSegment += len;
+ }
+ else {
+ if (remaining == 0) {
+ advance();
+ remaining = this.segmentSize - this.positionInSegment;
+ }
+ while (true) {
+ int toPut = Math.min(remaining, len);
+ this.currentSegment.put(this.positionInSegment, b, off, toPut);
+ off += toPut;
+ len -= toPut;
+
+ if (len > 0) {
+ this.positionInSegment = this.segmentSize;
+ advance();
+ remaining = this.segmentSize - this.positionInSegment;
+ }
+ else {
+ this.positionInSegment += toPut;
+ break;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void writeBoolean(boolean v) throws IOException {
+ writeByte(v ? 1 : 0);
+ }
+
+ @Override
+ public void writeByte(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize) {
+ this.currentSegment.put(this.positionInSegment++, (byte) v);
+ }
+ else {
+ advance();
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeShort(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 1) {
+ this.currentSegment.putShortBigEndian(this.positionInSegment, (short) v);
+ this.positionInSegment += 2;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeShort(v);
+ }
+ else {
+ writeByte(v >> 8);
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeChar(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 1) {
+ this.currentSegment.putCharBigEndian(this.positionInSegment, (char) v);
+ this.positionInSegment += 2;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeChar(v);
+ }
+ else {
+ writeByte(v >> 8);
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeInt(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 3) {
+ this.currentSegment.putIntBigEndian(this.positionInSegment, v);
+ this.positionInSegment += 4;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeInt(v);
+ }
+ else {
+ writeByte(v >> 24);
+ writeByte(v >> 16);
+ writeByte(v >> 8);
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeLong(long v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 7) {
+ this.currentSegment.putLongBigEndian(this.positionInSegment, v);
+ this.positionInSegment += 8;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeLong(v);
+ }
+ else {
+ writeByte((int) (v >> 56));
+ writeByte((int) (v >> 48));
+ writeByte((int) (v >> 40));
+ writeByte((int) (v >> 32));
+ writeByte((int) (v >> 24));
+ writeByte((int) (v >> 16));
+ writeByte((int) (v >> 8));
+ writeByte((int) v);
+ }
+ }
+
+ @Override
+ public void writeFloat(float v) throws IOException {
+ writeInt(Float.floatToRawIntBits(v));
+ }
+
+ @Override
+ public void writeDouble(double v) throws IOException {
+ writeLong(Double.doubleToRawLongBits(v));
+ }
+
+ @Override
+ public void writeBytes(String s) throws IOException {
+ for (int i = 0; i < s.length(); i++) {
+ writeByte(s.charAt(i));
+ }
+ }
+
+ @Override
+ public void writeChars(String s) throws IOException {
+ for (int i = 0; i < s.length(); i++) {
+ writeChar(s.charAt(i));
+ }
+ }
+
+ @Override
+ public void writeUTF(String str) throws IOException {
+ int strlen = str.length();
+ int utflen = 0;
+ int c, count = 0;
+
+ /* use charAt instead of copying String to char array */
+ for (int i = 0; i < strlen; i++) {
+ c = str.charAt(i);
+ if ((c >= 0x0001) && (c <= 0x007F)) {
+ utflen++;
+ } else if (c > 0x07FF) {
+ utflen += 3;
+ } else {
+ utflen += 2;
+ }
+ }
+
+ if (utflen > 65535) {
+ throw new UTFDataFormatException("encoded string too long: " + utflen + " memory");
+ }
+
+ if (this.utfBuffer == null || this.utfBuffer.length < utflen + 2) {
+ this.utfBuffer = new byte[utflen + 2];
+ }
+ final byte[] bytearr = this.utfBuffer;
+
+ bytearr[count++] = (byte) ((utflen >>> 8) & 0xFF);
+ bytearr[count++] = (byte) (utflen & 0xFF);
+
+ int i = 0;
+ for (i = 0; i < strlen; i++) {
+ c = str.charAt(i);
+ if (!((c >= 0x0001) && (c <= 0x007F))) {
+ break;
+ }
+ bytearr[count++] = (byte) c;
+ }
+
+ for (; i < strlen; i++) {
+ c = str.charAt(i);
+ if ((c >= 0x0001) && (c <= 0x007F)) {
+ bytearr[count++] = (byte) c;
+
+ } else if (c > 0x07FF) {
+ bytearr[count++] = (byte) (0xE0 | ((c >> 12) & 0x0F));
+ bytearr[count++] = (byte) (0x80 | ((c >> 6) & 0x3F));
+ bytearr[count++] = (byte) (0x80 | (c & 0x3F));
+ } else {
+ bytearr[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F));
+ bytearr[count++] = (byte) (0x80 | (c & 0x3F));
+ }
+ }
+
+ write(bytearr, 0, utflen + 2);
+ }
+
+ @Override
+ public void skipBytesToWrite(int numBytes) throws IOException {
+ while (numBytes > 0) {
+ final int remaining = this.segmentSize - this.positionInSegment;
+ if (numBytes <= remaining) {
+ this.positionInSegment += numBytes;
+ return;
+ }
+ this.positionInSegment = this.segmentSize;
+ advance();
+ numBytes -= remaining;
+ }
+ }
+
+ @Override
+ public void write(DataInputView source, int numBytes) throws IOException {
+ while (numBytes > 0) {
+ final int remaining = this.segmentSize - this.positionInSegment;
+ if (numBytes <= remaining) {
+ this.currentSegment.put(source, this.positionInSegment, numBytes);
+ this.positionInSegment += numBytes;
+ return;
+ }
+
+ if (remaining > 0) {
+ this.currentSegment.put(source, this.positionInSegment, remaining);
+ this.positionInSegment = this.segmentSize;
+ numBytes -= remaining;
+ }
+
+ advance();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/StringSerializationSpeedBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/StringSerializationSpeedBenchmark.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/StringSerializationSpeedBenchmark.java
new file mode 100644
index 0000000..2163eb5
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/StringSerializationSpeedBenchmark.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory.benchmarks;
+
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.core.memory.HybridMemorySegment;
+import org.apache.flink.core.memory.MemorySegment;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Random;
+
+public class StringSerializationSpeedBenchmark {
+
+ public static void main(String[] args) throws Exception {
+
+ final int LARGE_SEGMENT_SIZE = 1024 * 1024 * 1024;
+
+ final byte[] largeSegment = new byte[LARGE_SEGMENT_SIZE];
+
+ final ByteBuffer largeOffHeap = ByteBuffer.allocateDirect(LARGE_SEGMENT_SIZE);
+
+ final String[] randomStrings = generateRandomStrings(5468917685263896L, 1000, 128, 6, true);
+
+ final StringSerializer ser = StringSerializer.INSTANCE;
+
+ final int outerRounds = 10;
+ final int innerRounds = 5000;
+
+ {
+ System.out.println("testing core heap memory segment");
+
+ long start = System.nanoTime();
+ for (int outer = 0; outer < outerRounds; outer++) {
+
+ ArrayList<MemorySegment> memory = new ArrayList<>();
+ memory.add(HeapMemorySegment.FACTORY.wrapPooledHeapMemory(largeSegment, null));
+ ArrayList<MemorySegment> target = new ArrayList<>();
+
+ CoreMemorySegmentOutView output = new CoreMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+ for (int i = 0; i < innerRounds; i++) {
+ for (String s : randomStrings) {
+ ser.serialize(s, output);
+ }
+ }
+ }
+ long stop = System.nanoTime();
+
+ System.out.println(String.format("Core heap memory segment took %,d msecs", (stop - start) / 1000000));
+ }
+
+ {
+ System.out.println("testing core hybrid memory segment on heap");
+
+ long start = System.nanoTime();
+ for (int outer = 0; outer < outerRounds; outer++) {
+
+ ArrayList<MemorySegment> memory = new ArrayList<>();
+ memory.add(HybridMemorySegment.FACTORY.wrapPooledHeapMemory(largeSegment, null));
+ ArrayList<MemorySegment> target = new ArrayList<>();
+
+ CoreMemorySegmentOutView output = new CoreMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+ for (int i = 0; i < innerRounds; i++) {
+ for (String s : randomStrings) {
+ ser.serialize(s, output);
+ }
+ }
+ }
+ long stop = System.nanoTime();
+
+ System.out.println(String.format("Core hybrid memory segment on heap took %,d msecs", (stop - start) / 1000000));
+ }
+
+ {
+ System.out.println("testing core hybrid memory segment off heap");
+
+ long start = System.nanoTime();
+ for (int outer = 0; outer < outerRounds; outer++) {
+
+ ArrayList<MemorySegment> memory = new ArrayList<>();
+ memory.add(HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(largeOffHeap, null));
+ ArrayList<MemorySegment> target = new ArrayList<>();
+
+ CoreMemorySegmentOutView output = new CoreMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+ for (int i = 0; i < innerRounds; i++) {
+ for (String s : randomStrings) {
+ ser.serialize(s, output);
+ }
+ }
+ }
+ long stop = System.nanoTime();
+
+ System.out.println(String.format("Core hybrid memory segment off heap took %,d msecs", (stop - start) / 1000000));
+ }
+
+ {
+ System.out.println("testing pure hybrid memory segment on heap");
+
+ long start = System.nanoTime();
+ for (int outer = 0; outer < outerRounds; outer++) {
+
+ ArrayList<PureHybridMemorySegment> memory = new ArrayList<>();
+ memory.add(new PureHybridMemorySegment(largeSegment));
+ ArrayList<PureHybridMemorySegment> target = new ArrayList<>();
+
+ PureHybridMemorySegmentOutView output = new PureHybridMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+ for (int i = 0; i < innerRounds; i++) {
+ for (String s : randomStrings) {
+ ser.serialize(s, output);
+ }
+ }
+ }
+ long stop = System.nanoTime();
+
+ System.out.println(String.format("Pure hybrid on heap memory segment took %,d msecs", (stop - start) / 1000000));
+ }
+
+ {
+ System.out.println("testing pure hybrid memory segment off heap");
+
+ long start = System.nanoTime();
+ for (int outer = 0; outer < outerRounds; outer++) {
+
+ ArrayList<PureHybridMemorySegment> memory = new ArrayList<>();
+ memory.add(new PureHybridMemorySegment(largeOffHeap));
+ ArrayList<PureHybridMemorySegment> target = new ArrayList<>();
+
+ PureHybridMemorySegmentOutView output = new PureHybridMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+ for (int i = 0; i < innerRounds; i++) {
+ for (String s : randomStrings) {
+ ser.serialize(s, output);
+ }
+ }
+ }
+ long stop = System.nanoTime();
+
+ System.out.println(String.format("Pure hybrid off heap memory segment took %,d msecs", (stop - start) / 1000000));
+ }
+
+ {
+ System.out.println("testing pure heap memory segment");
+
+ long start = System.nanoTime();
+ for (int outer = 0; outer < outerRounds; outer++) {
+
+ ArrayList<PureHeapMemorySegment> memory = new ArrayList<>();
+ memory.add(new PureHeapMemorySegment(largeSegment));
+ ArrayList<PureHeapMemorySegment> target = new ArrayList<>();
+
+ PureHeapMemorySegmentOutView output = new PureHeapMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+ for (int i = 0; i < innerRounds; i++) {
+ for (String s : randomStrings) {
+ ser.serialize(s, output);
+ }
+ }
+ }
+ long stop = System.nanoTime();
+
+ System.out.println(String.format("Pure heap memory segment took %,d msecs", (stop - start) / 1000000));
+ }
+ }
+
+ private static String[] generateRandomStrings(long seed, int num, int maxLen, int minLen, boolean asciiOnly) {
+ Random rnd = new Random(seed);
+ String[] array = new String[num];
+ StringBuilder bld = new StringBuilder(maxLen);
+
+ int minCharValue = 40;
+ int charRange = asciiOnly ? 60 : 30000;
+
+ for (int i = 0; i < num; i++) {
+ bld.setLength(0);
+ int len = rnd.nextInt(maxLen - minLen) + minLen;
+
+ for (int k = 0; k < len; k++) {
+ bld.append((char) (rnd.nextInt(charRange) + minCharValue));
+ }
+
+ array[i] = bld.toString();
+ }
+
+ return array;
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/types/NormalizableKeyTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/types/NormalizableKeyTest.java b/flink-core/src/test/java/org/apache/flink/types/NormalizableKeyTest.java
index b39fb2e..d6c15d9 100644
--- a/flink-core/src/test/java/org/apache/flink/types/NormalizableKeyTest.java
+++ b/flink-core/src/test/java/org/apache/flink/types/NormalizableKeyTest.java
@@ -16,18 +16,12 @@
* limitations under the License.
*/
-
package org.apache.flink.types;
-import org.junit.Assert;
-
import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.types.CharValue;
-import org.apache.flink.types.IntValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.NormalizableKey;
-import org.apache.flink.types.NullValue;
-import org.apache.flink.types.StringValue;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+
+import org.junit.Assert;
import org.junit.Test;
public class NormalizableKeyTest {
@@ -124,9 +118,9 @@ public class NormalizableKeyTest {
for (int i = 0; i < 5; i++) {
// self checks
- for (int x = 0; x < allChars.length; x++) {
- for (int y = 0; y < allChars.length; y++) {
- assertNormalizableKey(allChars[x], allChars[y], i);
+ for (CharValue allChar1 : allChars) {
+ for (CharValue allChar : allChars) {
+ assertNormalizableKey(allChar1, allChar, i);
}
}
}
@@ -135,8 +129,8 @@ public class NormalizableKeyTest {
@SuppressWarnings("unchecked")
private <T extends Comparable<T>> void assertNormalizableKey(NormalizableKey<T> key1, NormalizableKey<T> key2, int len) {
- byte[] normalizedKeys = new byte[2*len];
- MemorySegment wrapper = new MemorySegment(normalizedKeys);
+ byte[] normalizedKeys = new byte[32];
+ MemorySegment wrapper = MemorySegmentFactory.wrap(normalizedKeys);
key1.copyNormalizedKey(wrapper, 0, len);
key2.copyNormalizedKey(wrapper, len, len);
@@ -147,13 +141,13 @@ public class NormalizableKeyTest {
int normKey2 = normalizedKeys[len + i] & 0xFF;
if ((comp = (normKey1 - normKey2)) != 0) {
- if (Math.signum(((T) key1).compareTo((T) key2)) != Math.signum(comp)) {
+ if (Math.signum(key1.compareTo((T) key2)) != Math.signum(comp)) {
Assert.fail("Normalized key comparison differs from actual key comparision");
}
return;
}
}
- if (((T) key1).compareTo((T) key2) != 0 && key1.getMaxNormalizedKeyLen() <= len) {
+ if (key1.compareTo((T) key2) != 0 && key1.getMaxNormalizedKeyLen() <= len) {
Assert.fail("Normalized key was not able to distinguish keys, " +
"although it should as the length of it sufficies to uniquely identify them");
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
index c742ce5..87d89eb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
@@ -29,7 +29,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.state.StateHandle;
import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java
index 736c245..2ca7f78 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java
@@ -27,8 +27,8 @@ import java.util.List;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.util.MathUtils;
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java
index b6c500f..8415d09 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java
@@ -26,8 +26,8 @@ import java.util.List;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.MemoryManager;
/**
* A {@link org.apache.flink.core.memory.DataOutputView} that is backed by a {@link BlockChannelWriter}, making it effectively a data output
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessInputView.java
index 718b8af..23dccb0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessInputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessInputView.java
@@ -23,7 +23,7 @@ import java.util.ArrayList;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.core.memory.SeekableDataInputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
import org.apache.flink.runtime.util.MathUtils;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessOutputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessOutputView.java
index 6ed9f4a..427fe84 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessOutputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessOutputView.java
@@ -23,7 +23,7 @@ import java.io.EOFException;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.core.memory.SeekableDataOutputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
import org.apache.flink.runtime.util.MathUtils;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java
index 7d8d485..355b2eb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java
@@ -28,8 +28,8 @@ import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader;
import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.util.MathUtils;
/**
[08/15] flink git commit: [FLINK-1320] [core] Add an off-heap variant
of the managed memory
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedInputView.java
deleted file mode 100644
index 23081c9..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedInputView.java
+++ /dev/null
@@ -1,566 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.runtime.memorymanager;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.UTFDataFormatException;
-
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.MemorySegment;
-
-
-/**
- * The base class for all input views that are backed by multiple memory pages. This base class contains all
- * decoding methods to read data from a page and detect page boundary crossing. The concrete sub classes must
- * implement the methods to provide the next memory page once the boundary is crossed.
- */
-public abstract class AbstractPagedInputView implements DataInputView {
-
- private MemorySegment currentSegment;
-
- protected final int headerLength; // the number of bytes to skip at the beginning of each segment
-
- private int positionInSegment; // the offset in the current segment
-
- private int limitInSegment; // the limit in the current segment before switching to the next
-
- private byte[] utfByteBuffer; // reusable byte buffer for utf-8 decoding
- private char[] utfCharBuffer; // reusable char buffer for utf-8 decoding
-
-
- // --------------------------------------------------------------------------------------------
- // Constructors
- // --------------------------------------------------------------------------------------------
-
- /**
- * Creates a new view that starts with the given segment. The input starts directly after the header
- * of the given page. If the header size is zero, it starts at the beginning. The specified initial
- * limit describes up to which position data may be read from the current segment, before the view must
- * advance to the next segment.
- *
- * @param initialSegment The memory segment to start reading from.
- * @param initialLimit The position one after the last valid byte in the initial segment.
- * @param headerLength The number of bytes to skip at the beginning of each segment for the header. This
- * length must be the same for all memory segments.
- */
- protected AbstractPagedInputView(MemorySegment initialSegment, int initialLimit, int headerLength) {
- this.headerLength = headerLength;
- this.positionInSegment = headerLength;
- seekInput(initialSegment, headerLength, initialLimit);
- }
-
- /**
- * Creates a new view that is initially not bound to a memory segment. This constructor is typically
- * for views that always seek first.
- * <p>
- * WARNING: The view is not readable until the first call to either {@link #advance()},
- * or to {@link #seekInput(MemorySegment, int, int)}.
- *
- * @param headerLength The number of bytes to skip at the beginning of each segment for the header.
- */
- protected AbstractPagedInputView(int headerLength) {
- this.headerLength = headerLength;
- }
-
- // --------------------------------------------------------------------------------------------
- // Page Management
- // --------------------------------------------------------------------------------------------
-
- /**
- * Gets the memory segment that will be used to read the next bytes from. If the segment is exactly exhausted,
- * meaning that the last byte read was the last byte available in the segment, then this segment will
- * not serve the next bytes. The segment to serve the next bytes will be obtained through the
- * {@link #nextSegment(MemorySegment)} method.
- *
- * @return The current memory segment.
- */
- public MemorySegment getCurrentSegment() {
- return this.currentSegment;
- }
-
- /**
- * Gets the position from which the next byte will be read. If that position is equal to the current limit,
- * then the next byte will be read from next segment.
- *
- * @return The position from which the next byte will be read.
- * @see #getCurrentSegmentLimit()
- */
- public int getCurrentPositionInSegment() {
- return this.positionInSegment;
- }
-
- /**
- * Gets the current limit in the memory segment. This value points to the byte one after the last valid byte
- * in the memory segment.
- *
- * @return The current limit in the memory segment.
- * @see #getCurrentPositionInSegment()
- */
- public int getCurrentSegmentLimit() {
- return this.limitInSegment;
- }
-
- /**
- * The method by which concrete subclasses realize page crossing. This method is invoked when the current page
- * is exhausted and a new page is required to continue the reading. If no further page is available, this
- * method must throw an {@link EOFException}.
- *
- * @param current The current page that was read to its limit. May be {@code null}, if this method is
- * invoked for the first time.
- * @return The next page from which the reading should continue. May not be {@code null}. If the input is
- * exhausted, an {@link EOFException} must be thrown instead.
- *
- * @throws EOFException Thrown, if no further segment is available.
- * @throws IOException Thrown, if the method cannot provide the next page due to an I/O related problem.
- */
- protected abstract MemorySegment nextSegment(MemorySegment current) throws EOFException, IOException;
-
- /**
- * Gets the limit for reading bytes from the given memory segment. This method must return the position
- * of the byte after the last valid byte in the given memory segment. When the position returned by this
- * method is reached, the view will attempt to switch to the next memory segment.
- *
- * @param segment The segment to determine the limit for.
- * @return The limit for the given memory segment.
- */
- protected abstract int getLimitForSegment(MemorySegment segment);
-
- /**
- * Advances the view to the next memory segment. The reading will continue after the header of the next
- * segment. This method uses {@link #nextSegment(MemorySegment)} and {@link #getLimitForSegment(MemorySegment)}
- * to get the next segment and set its limit.
- *
- * @throws IOException Thrown, if the next segment could not be obtained.
- *
- * @see #nextSegment(MemorySegment)
- * @see #getLimitForSegment(MemorySegment)
- */
- protected final void advance() throws IOException {
- // note: this code ensures that in case of EOF, we stay at the same position such that
- // EOF is reproducible (if nextSegment throws a reproducible EOFException)
- this.currentSegment = nextSegment(this.currentSegment);
- this.limitInSegment = getLimitForSegment(this.currentSegment);
- this.positionInSegment = this.headerLength;
- }
-
- /**
- * Sets the internal state of the view such that the next bytes will be read from the given memory segment,
- * starting at the given position. The memory segment will provide bytes up to the given limit position.
- *
- * @param segment The segment to read the next bytes from.
- * @param positionInSegment The position in the segment to start reading from.
- * @param limitInSegment The limit in the segment. When reached, the view will attempt to switch to
- * the next segment.
- */
- protected void seekInput(MemorySegment segment, int positionInSegment, int limitInSegment) {
- this.currentSegment = segment;
- this.positionInSegment = positionInSegment;
- this.limitInSegment = limitInSegment;
- }
-
- /**
- * Clears the internal state of the view. After this call, all read attempts will fail, until the
- * {@link #advance()} or {@link #seekInput(MemorySegment, int, int)} method have been invoked.
- */
- protected void clear() {
- this.currentSegment = null;
- this.positionInSegment = this.headerLength;
- this.limitInSegment = headerLength;
- }
-
- // --------------------------------------------------------------------------------------------
- // Data Input Specific methods
- // --------------------------------------------------------------------------------------------
-
- @Override
- public int read(byte[] b) throws IOException{
- return read(b,0,b.length);
- }
-
- @Override
- public int read(byte[] b, int off, int len) throws IOException{
- if (off < 0 || len < 0 || off + len > b.length) {
- throw new IndexOutOfBoundsException();
- }
-
- int remaining = this.limitInSegment - this.positionInSegment;
- if (remaining >= len) {
- this.currentSegment.get(this.positionInSegment, b, off, len);
- this.positionInSegment += len;
- return len;
- }
- else {
- if (remaining == 0) {
- try {
- advance();
- }catch(EOFException eof){
- return -1;
- }
- remaining = this.limitInSegment - this.positionInSegment;
- }
-
- int bytesRead = 0;
- while (true) {
- int toRead = Math.min(remaining, len-bytesRead);
- this.currentSegment.get(this.positionInSegment, b, off, toRead);
- off += toRead;
- bytesRead += toRead;
-
- if (len > bytesRead) {
- try {
- advance();
- }catch(EOFException eof){
- this.positionInSegment += toRead;
- return bytesRead;
- }
- remaining = this.limitInSegment - this.positionInSegment;
- }
- else {
- this.positionInSegment += toRead;
- break;
- }
- }
- return len;
- }
- }
-
- @Override
- public void readFully(byte[] b) throws IOException {
- readFully(b, 0, b.length);
- }
-
- @Override
- public void readFully(byte[] b, int off, int len) throws IOException {
- int bytesRead = read(b,off,len);
-
- if(bytesRead < len){
- throw new EOFException("There is no enough data left in the DataInputView.");
- }
- }
-
- @Override
- public boolean readBoolean() throws IOException {
- return readByte() == 1;
- }
-
- @Override
- public byte readByte() throws IOException {
- if (this.positionInSegment < this.limitInSegment) {
- return this.currentSegment.get(this.positionInSegment++);
- }
- else {
- advance();
- return readByte();
- }
- }
-
- @Override
- public int readUnsignedByte() throws IOException {
- return readByte() & 0xff;
- }
-
- @Override
- public short readShort() throws IOException {
- if (this.positionInSegment < this.limitInSegment - 1) {
- final short v = this.currentSegment.getShort(this.positionInSegment);
- this.positionInSegment += 2;
- return v;
- }
- else if (this.positionInSegment == this.limitInSegment) {
- advance();
- return readShort();
- }
- else {
- return (short) ((readUnsignedByte() << 8) | readUnsignedByte());
- }
- }
-
- @Override
- public int readUnsignedShort() throws IOException {
- if (this.positionInSegment < this.limitInSegment - 1) {
- final int v = this.currentSegment.getShort(this.positionInSegment) & 0xffff;
- this.positionInSegment += 2;
- return v;
- }
- else if (this.positionInSegment == this.limitInSegment) {
- advance();
- return readUnsignedShort();
- }
- else {
- return (readUnsignedByte() << 8) | readUnsignedByte();
- }
- }
-
- @Override
- public char readChar() throws IOException {
- if (this.positionInSegment < this.limitInSegment - 1) {
- final char v = this.currentSegment.getChar(this.positionInSegment);
- this.positionInSegment += 2;
- return v;
- }
- else if (this.positionInSegment == this.limitInSegment) {
- advance();
- return readChar();
- }
- else {
- return (char) ((readUnsignedByte() << 8) | readUnsignedByte());
- }
- }
-
- @Override
- public int readInt() throws IOException {
- if (this.positionInSegment < this.limitInSegment - 3) {
- final int v = this.currentSegment.getIntBigEndian(this.positionInSegment);
- this.positionInSegment += 4;
- return v;
- }
- else if (this.positionInSegment == this.limitInSegment) {
- advance();
- return readInt();
- }
- else {
- return (readUnsignedByte() << 24) |
- (readUnsignedByte() << 16) |
- (readUnsignedByte() << 8) |
- readUnsignedByte();
- }
- }
-
- @Override
- public long readLong() throws IOException {
- if (this.positionInSegment < this.limitInSegment - 7) {
- final long v = this.currentSegment.getLongBigEndian(this.positionInSegment);
- this.positionInSegment += 8;
- return v;
- }
- else if (this.positionInSegment == this.limitInSegment) {
- advance();
- return readLong();
- }
- else {
- long l = 0L;
- l |= ((long) readUnsignedByte()) << 56;
- l |= ((long) readUnsignedByte()) << 48;
- l |= ((long) readUnsignedByte()) << 40;
- l |= ((long) readUnsignedByte()) << 32;
- l |= ((long) readUnsignedByte()) << 24;
- l |= ((long) readUnsignedByte()) << 16;
- l |= ((long) readUnsignedByte()) << 8;
- l |= (long) readUnsignedByte();
- return l;
- }
- }
-
- @Override
- public float readFloat() throws IOException {
- return Float.intBitsToFloat(readInt());
- }
-
- @Override
- public double readDouble() throws IOException {
- return Double.longBitsToDouble(readLong());
- }
-
- @Override
- public String readLine() throws IOException {
- final StringBuilder bld = new StringBuilder(32);
-
- try {
- int b;
- while ((b = readUnsignedByte()) != '\n') {
- if (b != '\r') {
- bld.append((char) b);
- }
- }
- }
- catch (EOFException eofex) {}
-
- if (bld.length() == 0) {
- return null;
- }
-
- // trim a trailing carriage return
- int len = bld.length();
- if (len > 0 && bld.charAt(len - 1) == '\r') {
- bld.setLength(len - 1);
- }
- return bld.toString();
- }
-
- @Override
- public String readUTF() throws IOException {
- final int utflen = readUnsignedShort();
-
- final byte[] bytearr;
- final char[] chararr;
-
- if (this.utfByteBuffer == null || this.utfByteBuffer.length < utflen) {
- bytearr = new byte[utflen];
- this.utfByteBuffer = bytearr;
- } else {
- bytearr = this.utfByteBuffer;
- }
- if (this.utfCharBuffer == null || this.utfCharBuffer.length < utflen) {
- chararr = new char[utflen];
- this.utfCharBuffer = chararr;
- } else {
- chararr = this.utfCharBuffer;
- }
-
- int c, char2, char3;
- int count = 0;
- int chararr_count = 0;
-
- readFully(bytearr, 0, utflen);
-
- while (count < utflen) {
- c = (int) bytearr[count] & 0xff;
- if (c > 127) {
- break;
- }
- count++;
- chararr[chararr_count++] = (char) c;
- }
-
- while (count < utflen) {
- c = (int) bytearr[count] & 0xff;
- switch (c >> 4) {
- case 0:
- case 1:
- case 2:
- case 3:
- case 4:
- case 5:
- case 6:
- case 7:
- /* 0xxxxxxx */
- count++;
- chararr[chararr_count++] = (char) c;
- break;
- case 12:
- case 13:
- /* 110x xxxx 10xx xxxx */
- count += 2;
- if (count > utflen) {
- throw new UTFDataFormatException("malformed input: partial character at end");
- }
- char2 = (int) bytearr[count - 1];
- if ((char2 & 0xC0) != 0x80) {
- throw new UTFDataFormatException("malformed input around byte " + count);
- }
- chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
- break;
- case 14:
- /* 1110 xxxx 10xx xxxx 10xx xxxx */
- count += 3;
- if (count > utflen) {
- throw new UTFDataFormatException("malformed input: partial character at end");
- }
- char2 = (int) bytearr[count - 2];
- char3 = (int) bytearr[count - 1];
- if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
- throw new UTFDataFormatException("malformed input around byte " + (count - 1));
- }
- chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
- break;
- default:
- /* 10xx xxxx, 1111 xxxx */
- throw new UTFDataFormatException("malformed input around byte " + count);
- }
- }
- // The number of chars produced may be less than utflen
- return new String(chararr, 0, chararr_count);
- }
-
- @Override
- public int skipBytes(int n) throws IOException {
- if (n < 0) {
- throw new IllegalArgumentException();
- }
-
- int remaining = this.limitInSegment - this.positionInSegment;
- if (remaining >= n) {
- this.positionInSegment += n;
- return n;
- }
- else {
- if (remaining == 0) {
- try {
- advance();
- } catch (EOFException eofex) {
- return 0;
- }
- remaining = this.limitInSegment - this.positionInSegment;
- }
-
- int skipped = 0;
- while (true) {
- int toSkip = Math.min(remaining, n);
- n -= toSkip;
- skipped += toSkip;
-
- if (n > 0) {
- try {
- advance();
- } catch (EOFException eofex) {
- return skipped;
- }
- remaining = this.limitInSegment - this.positionInSegment;
- }
- else {
- this.positionInSegment += toSkip;
- break;
- }
- }
- return skipped;
- }
- }
-
- @Override
- public void skipBytesToRead(int numBytes) throws IOException {
- if (numBytes < 0) {
- throw new IllegalArgumentException();
- }
-
- int remaining = this.limitInSegment - this.positionInSegment;
- if (remaining >= numBytes) {
- this.positionInSegment += numBytes;
- }
- else {
- if (remaining == 0) {
- advance();
- remaining = this.limitInSegment - this.positionInSegment;
- }
-
- while (true) {
- if (numBytes > remaining) {
- numBytes -= remaining;
- advance();
- remaining = this.limitInSegment - this.positionInSegment;
- }
- else {
- this.positionInSegment += numBytes;
- break;
- }
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedOutputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedOutputView.java
deleted file mode 100644
index ea73e62..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedOutputView.java
+++ /dev/null
@@ -1,416 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.runtime.memorymanager;
-
-import java.io.IOException;
-import java.io.UTFDataFormatException;
-
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.MemorySegment;
-
-
-/**
- * The base class for all output views that are backed by multiple memory pages. This base class contains all
- * encoding methods to write data to a page and detect page boundary crossing. The concrete sub classes must
- * implement the methods to collect the current page and provide the next memory page once the boundary is crossed.
- * <p>
- * The paging assumes that all memory segments are of the same size.
- */
-public abstract class AbstractPagedOutputView implements DataOutputView {
-
- private MemorySegment currentSegment; // the current memory segment to write to
-
- protected final int segmentSize; // the size of the memory segments
-
- protected final int headerLength; // the number of bytes to skip at the beginning of each segment
-
- private int positionInSegment; // the offset in the current segment
-
- private byte[] utfBuffer; // the reusable array for UTF encodings
-
-
- // --------------------------------------------------------------------------------------------
- // Constructors
- // --------------------------------------------------------------------------------------------
-
- /**
- * Creates a new output view that writes initially to the given initial segment. All segments in the
- * view have to be of the given {@code segmentSize}. A header of length {@code headerLength} is left
- * at the beginning of each segment.
- *
- * @param initialSegment The segment that the view starts writing to.
- * @param segmentSize The size of the memory segments.
- * @param headerLength The number of bytes to skip at the beginning of each segment for the header.
- */
- protected AbstractPagedOutputView(MemorySegment initialSegment, int segmentSize, int headerLength) {
- if (initialSegment == null) {
- throw new NullPointerException("Initial Segment may not be null");
- }
- this.segmentSize = segmentSize;
- this.headerLength = headerLength;
- this.currentSegment = initialSegment;
- this.positionInSegment = headerLength;
- }
-
- /**
- * @param segmentSize The size of the memory segments.
- * @param headerLength The number of bytes to skip at the beginning of each segment for the header.
- */
- protected AbstractPagedOutputView(int segmentSize, int headerLength)
- {
- this.segmentSize = segmentSize;
- this.headerLength = headerLength;
- }
-
-
- // --------------------------------------------------------------------------------------------
- // Page Management
- // --------------------------------------------------------------------------------------------
-
- /**
- *
- * This method must return a segment. If no more segments are available, it must throw an
- * {@link java.io.EOFException}.
- *
- * @param current The current memory segment
- * @param positionInCurrent The position in the segment, one after the last valid byte.
- * @return The next memory segment.
- *
- * @throws IOException
- */
- protected abstract MemorySegment nextSegment(MemorySegment current, int positionInCurrent) throws IOException;
-
-
- /**
- * Gets the segment that the view currently writes to.
- *
- * @return The segment the view currently writes to.
- */
- public MemorySegment getCurrentSegment() {
- return this.currentSegment;
- }
-
- /**
- * Gets the current write position (the position where the next bytes will be written)
- * in the current memory segment.
- *
- * @return The current write offset in the current memory segment.
- */
- public int getCurrentPositionInSegment() {
- return this.positionInSegment;
- }
-
- /**
- * Gets the size of the segments used by this view.
- *
- * @return The memory segment size.
- */
- public int getSegmentSize() {
- return this.segmentSize;
- }
-
- /**
- * Moves the output view to the next page. This method invokes internally the
- * {@link #nextSegment(MemorySegment, int)} method to give the current memory segment to the concrete subclass'
- * implementation and obtain the next segment to write to. Writing will continue inside the new segment
- * after the header.
- *
- * @throws IOException Thrown, if the current segment could not be processed or a new segment could not
- * be obtained.
- */
- protected void advance() throws IOException {
- this.currentSegment = nextSegment(this.currentSegment, this.positionInSegment);
- this.positionInSegment = this.headerLength;
- }
-
- /**
- * Sets the internal state to the given memory segment and the given position within the segment.
- *
- * @param seg The memory segment to write the next bytes to.
- * @param position The position to start writing the next bytes to.
- */
- protected void seekOutput(MemorySegment seg, int position) {
- this.currentSegment = seg;
- this.positionInSegment = position;
- }
-
- /**
- * Clears the internal state. Any successive write calls will fail until either {@link #advance()} or
- * {@link #seekOutput(MemorySegment, int)} is called.
- *
- * @see #advance()
- * @see #seekOutput(MemorySegment, int)
- */
- protected void clear() {
- this.currentSegment = null;
- this.positionInSegment = this.headerLength;
- }
-
- // --------------------------------------------------------------------------------------------
- // Data Output Specific methods
- // --------------------------------------------------------------------------------------------
-
- @Override
- public void write(int b) throws IOException {
- writeByte(b);
- }
-
- @Override
- public void write(byte[] b) throws IOException {
- write(b, 0, b.length);
- }
-
- @Override
- public void write(byte[] b, int off, int len) throws IOException {
- int remaining = this.segmentSize - this.positionInSegment;
- if (remaining >= len) {
- this.currentSegment.put(this.positionInSegment, b, off, len);
- this.positionInSegment += len;
- }
- else {
- if (remaining == 0) {
- advance();
- remaining = this.segmentSize - this.positionInSegment;
- }
- while (true) {
- int toPut = Math.min(remaining, len);
- this.currentSegment.put(this.positionInSegment, b, off, toPut);
- off += toPut;
- len -= toPut;
-
- if (len > 0) {
- this.positionInSegment = this.segmentSize;
- advance();
- remaining = this.segmentSize - this.positionInSegment;
- }
- else {
- this.positionInSegment += toPut;
- break;
- }
- }
- }
- }
-
- @Override
- public void writeBoolean(boolean v) throws IOException {
- writeByte(v ? 1 : 0);
- }
-
- @Override
- public void writeByte(int v) throws IOException {
- if (this.positionInSegment < this.segmentSize) {
- this.currentSegment.put(this.positionInSegment++, (byte) v);
- }
- else {
- advance();
- writeByte(v);
- }
- }
-
- @Override
- public void writeShort(int v) throws IOException {
- if (this.positionInSegment < this.segmentSize - 1) {
- this.currentSegment.putShort(this.positionInSegment, (short) v);
- this.positionInSegment += 2;
- }
- else if (this.positionInSegment == this.segmentSize) {
- advance();
- writeShort(v);
- }
- else {
- writeByte(v >> 8);
- writeByte(v);
- }
- }
-
- @Override
- public void writeChar(int v) throws IOException {
- if (this.positionInSegment < this.segmentSize - 1) {
- this.currentSegment.putChar(this.positionInSegment, (char) v);
- this.positionInSegment += 2;
- }
- else if (this.positionInSegment == this.segmentSize) {
- advance();
- writeChar(v);
- }
- else {
- writeByte(v >> 8);
- writeByte(v);
- }
- }
-
- @Override
- public void writeInt(int v) throws IOException {
- if (this.positionInSegment < this.segmentSize - 3) {
- this.currentSegment.putIntBigEndian(this.positionInSegment, v);
- this.positionInSegment += 4;
- }
- else if (this.positionInSegment == this.segmentSize) {
- advance();
- writeInt(v);
- }
- else {
- writeByte(v >> 24);
- writeByte(v >> 16);
- writeByte(v >> 8);
- writeByte(v);
- }
- }
-
- @Override
- public void writeLong(long v) throws IOException {
- if (this.positionInSegment < this.segmentSize - 7) {
- this.currentSegment.putLongBigEndian(this.positionInSegment, v);
- this.positionInSegment += 8;
- }
- else if (this.positionInSegment == this.segmentSize) {
- advance();
- writeLong(v);
- }
- else {
- writeByte((int) (v >> 56));
- writeByte((int) (v >> 48));
- writeByte((int) (v >> 40));
- writeByte((int) (v >> 32));
- writeByte((int) (v >> 24));
- writeByte((int) (v >> 16));
- writeByte((int) (v >> 8));
- writeByte((int) v);
- }
- }
-
- @Override
- public void writeFloat(float v) throws IOException {
- writeInt(Float.floatToRawIntBits(v));
- }
-
- @Override
- public void writeDouble(double v) throws IOException {
- writeLong(Double.doubleToRawLongBits(v));
- }
-
- @Override
- public void writeBytes(String s) throws IOException {
- for (int i = 0; i < s.length(); i++) {
- writeByte(s.charAt(i));
- }
- }
-
- @Override
- public void writeChars(String s) throws IOException {
- for (int i = 0; i < s.length(); i++) {
- writeChar(s.charAt(i));
- }
- }
-
- @Override
- public void writeUTF(String str) throws IOException {
- int strlen = str.length();
- int utflen = 0;
- int c, count = 0;
-
- /* use charAt instead of copying String to char array */
- for (int i = 0; i < strlen; i++) {
- c = str.charAt(i);
- if ((c >= 0x0001) && (c <= 0x007F)) {
- utflen++;
- } else if (c > 0x07FF) {
- utflen += 3;
- } else {
- utflen += 2;
- }
- }
-
- if (utflen > 65535) {
- throw new UTFDataFormatException("encoded string too long: " + utflen + " memory");
- }
-
- if (this.utfBuffer == null || this.utfBuffer.length < utflen + 2) {
- this.utfBuffer = new byte[utflen + 2];
- }
- final byte[] bytearr = this.utfBuffer;
-
- bytearr[count++] = (byte) ((utflen >>> 8) & 0xFF);
- bytearr[count++] = (byte) ((utflen >>> 0) & 0xFF);
-
- int i = 0;
- for (i = 0; i < strlen; i++) {
- c = str.charAt(i);
- if (!((c >= 0x0001) && (c <= 0x007F))) {
- break;
- }
- bytearr[count++] = (byte) c;
- }
-
- for (; i < strlen; i++) {
- c = str.charAt(i);
- if ((c >= 0x0001) && (c <= 0x007F)) {
- bytearr[count++] = (byte) c;
-
- } else if (c > 0x07FF) {
- bytearr[count++] = (byte) (0xE0 | ((c >> 12) & 0x0F));
- bytearr[count++] = (byte) (0x80 | ((c >> 6) & 0x3F));
- bytearr[count++] = (byte) (0x80 | ((c >> 0) & 0x3F));
- } else {
- bytearr[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F));
- bytearr[count++] = (byte) (0x80 | ((c >> 0) & 0x3F));
- }
- }
-
- write(bytearr, 0, utflen + 2);
- }
-
- @Override
- public void skipBytesToWrite(int numBytes) throws IOException {
- while (numBytes > 0) {
- final int remaining = this.segmentSize - this.positionInSegment;
- if (numBytes <= remaining) {
- this.positionInSegment += numBytes;
- return;
- }
- this.positionInSegment = this.segmentSize;
- advance();
- numBytes -= remaining;
- }
- return;
- }
-
- @Override
- public void write(DataInputView source, int numBytes) throws IOException {
- while (numBytes > 0) {
- final int remaining = this.segmentSize - this.positionInSegment;
- if (numBytes <= remaining) {
- this.currentSegment.put(source, this.positionInSegment, numBytes);
- this.positionInSegment += numBytes;
- return;
- }
-
- if (remaining > 0) {
- this.currentSegment.put(source, this.positionInSegment, remaining);
- this.positionInSegment = this.segmentSize;
- numBytes -= remaining;
- }
-
- advance();
- }
- return;
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/CheckedMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/CheckedMemorySegment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/CheckedMemorySegment.java
deleted file mode 100644
index 85c91c0..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/CheckedMemorySegment.java
+++ /dev/null
@@ -1,407 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.runtime.memorymanager;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-/**
- * This class represents a piece of memory allocated from the memory manager. The segment is backed
- * by a byte array and features random put and get methods for the basic types that are stored in a byte-wise
- * fashion in the memory.
- */
-public class CheckedMemorySegment {
-
- /**
- * The array in which the data is stored.
- */
- protected byte[] memory;
-
- /**
- * The offset in the memory array where this segment starts.
- */
- protected final int offset;
-
- /**
- * The size of the memory segment.
- */
- protected final int size;
-
- /**
- * Wrapper for I/O requests.
- */
- protected ByteBuffer wrapper;
-
- // -------------------------------------------------------------------------
- // Constructors
- // -------------------------------------------------------------------------
-
- public CheckedMemorySegment(byte[] memory) {
- this.memory = memory;
- this.offset = 0;
- this.size = memory.length;
- }
-
- // -------------------------------------------------------------------------
- // MemorySegment Accessors
- // -------------------------------------------------------------------------
-
- public boolean isFreed() {
- return this.memory == null;
- }
-
- public final int size() {
- return size;
- }
-
- public final byte[] getBackingArray() {
- return this.memory;
- }
-
- public final int translateOffset(int offset) {
- return this.offset + offset;
- }
-
- // -------------------------------------------------------------------------
- // Helper methods
- // -------------------------------------------------------------------------
-
- public ByteBuffer wrap(int offset, int length) {
- if (offset > this.size || offset > this.size - length) {
- throw new IndexOutOfBoundsException();
- }
-
- if (this.wrapper == null) {
- this.wrapper = ByteBuffer.wrap(this.memory, this.offset + offset, length);
- }
- else {
- this.wrapper.position(this.offset + offset);
- this.wrapper.limit(this.offset + offset + length);
- }
-
- return this.wrapper;
- }
-
-
- // --------------------------------------------------------------------
- // Random Access
- // --------------------------------------------------------------------
-
- // ------------------------------------------------------------------------------------------------------
- // WARNING: Any code for range checking must take care to avoid integer overflows. The position
- // integer may go up to <code>Integer.MAX_VALUE</tt>. Range checks that work after the principle
- // <code>position + 3 < end</code> may fail because <code>position + 3</code> becomes negative.
- // A safe solution is to subtract the delta from the limit, for example
- // <code>position < end - 3</code>. Since all indices are always positive, and the integer domain
- // has one more negative value than positive values, this can never cause an underflow.
- // ------------------------------------------------------------------------------------------------------
-
- public final byte get(int index) {
- if (index >= 0 && index < this.size) {
- return this.memory[this.offset + index];
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final CheckedMemorySegment put(int index, byte b) {
- if (index >= 0 && index < this.size) {
- this.memory[this.offset + index] = b;
- return this;
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final CheckedMemorySegment get(int index, byte[] dst) {
- return get(index, dst, 0, dst.length);
- }
-
- public final CheckedMemorySegment put(int index, byte[] src) {
- return put(index, src, 0, src.length);
- }
-
- public final CheckedMemorySegment get(int index, byte[] dst, int offset, int length) {
- if (index >= 0 && index < this.size && index <= this.size - length && offset <= dst.length - length) {
- System.arraycopy(this.memory, this.offset + index, dst, offset, length);
- return this;
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final CheckedMemorySegment put(int index, byte[] src, int offset, int length) {
- if (index >= 0 && index < this.size && index <= this.size - length && offset <= src.length - length) {
- System.arraycopy(src, offset, this.memory, this.offset + index, length);
- return this;
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final CheckedMemorySegment get(DataOutput out, int offset, int length) throws IOException {
- if (offset >= 0 && offset < this.size && length >= 0 && offset <= this.size - length) {
- out.write(this.memory, this.offset + offset, length);
- return this;
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final CheckedMemorySegment put(DataInput in, int offset, int length) throws IOException {
- if (offset >= 0 && offset < this.size && length >= 0 && offset <= this.size - length) {
- in.readFully(this.memory, this.offset + offset, length);
- return this;
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final boolean getBoolean(int index) {
- if (index >= 0 && index < this.size) {
- return this.memory[this.offset + index] != 0;
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final CheckedMemorySegment putBoolean(int index, boolean value) {
- if (index >= 0 && index < this.size) {
- this.memory[this.offset + index] = (byte) (value ? 1 : 0);
- return this;
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final char getChar(int index) {
- if (index >= 0 && index < this.size - 1) {
- return (char) ( ((this.memory[this.offset + index + 0] & 0xff) << 8) |
- (this.memory[this.offset + index + 1] & 0xff) );
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final CheckedMemorySegment putChar(int index, char value) {
- if (index >= 0 && index < this.size - 1) {
- this.memory[this.offset + index + 0] = (byte) (value >> 8);
- this.memory[this.offset + index + 1] = (byte) value;
- return this;
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final short getShort(int index) {
- if (index >= 0 && index < this.size - 1) {
- return (short) (
- ((this.memory[this.offset + index + 0] & 0xff) << 8) |
- ((this.memory[this.offset + index + 1] & 0xff)) );
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final CheckedMemorySegment putShort(int index, short value) {
- if (index >= 0 && index < this.size - 1) {
- this.memory[this.offset + index + 0] = (byte) (value >> 8);
- this.memory[this.offset + index + 1] = (byte) value;
- return this;
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final int getInt(int index) {
- if (index >= 0 && index < this.size - 3) {
- return ((this.memory[index ] & 0xff) << 24)
- | ((this.memory[index + 1] & 0xff) << 16)
- | ((this.memory[index + 2] & 0xff) << 8)
- | ((this.memory[index + 3] & 0xff) );
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final int getIntLittleEndian(int index) {
- if (index >= 0 && index < this.size - 3) {
- return ((this.memory[index ] & 0xff) )
- | ((this.memory[index + 1] & 0xff) << 8)
- | ((this.memory[index + 2] & 0xff) << 16)
- | ((this.memory[index + 3] & 0xff) << 24);
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final int getIntBigEndian(int index) {
- if (index >= 0 && index < this.size - 3) {
- return ((this.memory[index ] & 0xff) << 24)
- | ((this.memory[index + 1] & 0xff) << 16)
- | ((this.memory[index + 2] & 0xff) << 8)
- | ((this.memory[index + 3] & 0xff) );
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final void putInt(int index, int value) {
- if (index >= 0 && index < this.size - 3) {
- this.memory[index ] = (byte) (value >> 24);
- this.memory[index + 1] = (byte) (value >> 16);
- this.memory[index + 2] = (byte) (value >> 8);
- this.memory[index + 3] = (byte) value;
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final void putIntLittleEndian(int index, int value) {
- if (index >= 0 && index < this.size - 3) {
- this.memory[index ] = (byte) value;
- this.memory[index + 1] = (byte) (value >> 8);
- this.memory[index + 2] = (byte) (value >> 16);
- this.memory[index + 3] = (byte) (value >> 24);
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final void putIntBigEndian(int index, int value) {
- if (index >= 0 && index < this.size - 3) {
- this.memory[index ] = (byte) (value >> 24);
- this.memory[index + 1] = (byte) (value >> 16);
- this.memory[index + 2] = (byte) (value >> 8);
- this.memory[index + 3] = (byte) value;
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final long getLong(int index) {
- if (index >= 0 && index < this.size - 7) {
- return (((long) this.memory[index ] & 0xff) << 56)
- | (((long) this.memory[index + 1] & 0xff) << 48)
- | (((long) this.memory[index + 2] & 0xff) << 40)
- | (((long) this.memory[index + 3] & 0xff) << 32)
- | (((long) this.memory[index + 4] & 0xff) << 24)
- | (((long) this.memory[index + 5] & 0xff) << 16)
- | (((long) this.memory[index + 6] & 0xff) << 8)
- | (((long) this.memory[index + 7] & 0xff) );
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
- public final long getLongLittleEndian(int index) {
- if (index >= 0 && index < this.size - 7) {
- return (((long) this.memory[index ] & 0xff) )
- | (((long) this.memory[index + 1] & 0xff) << 8)
- | (((long) this.memory[index + 2] & 0xff) << 16)
- | (((long) this.memory[index + 3] & 0xff) << 24)
- | (((long) this.memory[index + 4] & 0xff) << 32)
- | (((long) this.memory[index + 5] & 0xff) << 40)
- | (((long) this.memory[index + 6] & 0xff) << 48)
- | (((long) this.memory[index + 7] & 0xff) << 56);
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final long getLongBigEndian(int index) {
- if (index >= 0 && index < this.size - 7) {
- return (((long) this.memory[index ] & 0xff) << 56)
- | (((long) this.memory[index + 1] & 0xff) << 48)
- | (((long) this.memory[index + 2] & 0xff) << 40)
- | (((long) this.memory[index + 3] & 0xff) << 32)
- | (((long) this.memory[index + 4] & 0xff) << 24)
- | (((long) this.memory[index + 5] & 0xff) << 16)
- | (((long) this.memory[index + 6] & 0xff) << 8)
- | (((long) this.memory[index + 7] & 0xff) );
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final void putLong(int index, long value) {
- if (index >= 0 && index < this.size - 7) {
- this.memory[index ] = (byte) (value >> 56);
- this.memory[index + 1] = (byte) (value >> 48);
- this.memory[index + 2] = (byte) (value >> 40);
- this.memory[index + 3] = (byte) (value >> 32);
- this.memory[index + 4] = (byte) (value >> 24);
- this.memory[index + 5] = (byte) (value >> 16);
- this.memory[index + 6] = (byte) (value >> 8);
- this.memory[index + 7] = (byte) value;
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final void putLongLittleEndian(int index, long value) {
- if (index >= 0 && index < this.size - 7) {
- this.memory[index ] = (byte) value;
- this.memory[index + 1] = (byte) (value >> 8);
- this.memory[index + 2] = (byte) (value >> 16);
- this.memory[index + 3] = (byte) (value >> 24);
- this.memory[index + 4] = (byte) (value >> 32);
- this.memory[index + 5] = (byte) (value >> 40);
- this.memory[index + 6] = (byte) (value >> 48);
- this.memory[index + 7] = (byte) (value >> 56);
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final void putLongBigEndian(int index, long value) {
- if (index >= 0 && index < this.size - 7) {
- this.memory[index ] = (byte) (value >> 56);
- this.memory[index + 1] = (byte) (value >> 48);
- this.memory[index + 2] = (byte) (value >> 40);
- this.memory[index + 3] = (byte) (value >> 32);
- this.memory[index + 4] = (byte) (value >> 24);
- this.memory[index + 5] = (byte) (value >> 16);
- this.memory[index + 6] = (byte) (value >> 8);
- this.memory[index + 7] = (byte) value;
- } else {
- throw new IndexOutOfBoundsException();
- }
- }
-
- public final float getFloat(int index) {
- return Float.intBitsToFloat(getInt(index));
- }
-
- public final CheckedMemorySegment putFloat(int index, float value) {
- putLong(index, Float.floatToIntBits(value));
- return this;
- }
-
- public final double getDouble(int index) {
- return Double.longBitsToDouble(getLong(index));
- }
-
- public final CheckedMemorySegment putDouble(int index, double value) {
- putLong(index, Double.doubleToLongBits(value));
- return this;
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/DefaultMemoryManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/DefaultMemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/DefaultMemoryManager.java
deleted file mode 100644
index b041ac9..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/DefaultMemoryManager.java
+++ /dev/null
@@ -1,490 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.runtime.memorymanager;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.ConcurrentModificationException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
-
-public class DefaultMemoryManager implements MemoryManager {
-
- /**
- * The default memory page size. Currently set to 32 KiBytes.
- */
- public static final int DEFAULT_PAGE_SIZE = 32 * 1024;
-
- /**
- * The minimal memory page size. Currently set to 4 KiBytes.
- */
- public static final int MIN_PAGE_SIZE = 4 * 1024;
-
- /**
- * The Logger.
- */
- private static final Logger LOG = LoggerFactory.getLogger(DefaultMemoryManager.class);
-
- // --------------------------------------------------------------------------------------------
-
- private final Object lock = new Object(); // The lock used on the shared structures.
-
- private final ArrayDeque<byte[]> freeSegments; // the free memory segments
-
- private final HashMap<AbstractInvokable, Set<DefaultMemorySegment>> allocatedSegments;
-
- private final long roundingMask; // mask used to round down sizes to multiples of the page size
-
- private final int pageSize; // the page size, in bytes
-
- private final int pageSizeBits; // the number of bits that the power-of-two page size corresponds to
-
- private final int totalNumPages; // The initial total size, for verification.
-
- /** The total size of the memory managed by this memory manager */
- private final long memorySize;
-
- /** Number of slots of the task manager */
- private final int numberOfSlots;
-
- private final boolean isPreAllocated;
-
- /** The number of memory pages that have not been allocated and are available for lazy allocation */
- private int numNonAllocatedPages;
-
- /** flag whether the close() has already been invoked */
- private boolean isShutDown;
-
- // ------------------------------------------------------------------------
- // Constructors / Destructors
- // ------------------------------------------------------------------------
-
- /**
- * Creates a memory manager with the given capacity, using the default page size.
- *
- * @param memorySize The total size of the memory to be managed by this memory manager.
- * @param numberOfSlots The number of slots of the task manager.
- */
- public DefaultMemoryManager(long memorySize, int numberOfSlots) {
- this(memorySize, numberOfSlots, DEFAULT_PAGE_SIZE, true);
- }
-
- /**
- * Creates a memory manager with the given capacity and given page size.
- *
- * @param memorySize The total size of the memory to be managed by this memory manager.
- * @param numberOfSlots The number of slots of the task manager.
- * @param pageSize The size of the pages handed out by the memory manager.
- * @param preAllocateMemory True, if the memory manaber should immediately allocate all memory, false
- * if it should allocate and release the memory as needed.
- */
- public DefaultMemoryManager(long memorySize, int numberOfSlots, int pageSize, boolean preAllocateMemory) {
- // sanity checks
- if (memorySize <= 0) {
- throw new IllegalArgumentException("Size of total memory must be positive.");
- }
- if (pageSize < MIN_PAGE_SIZE) {
- throw new IllegalArgumentException("The page size must be at least " + MIN_PAGE_SIZE + " bytes.");
- }
- if ((pageSize & (pageSize - 1)) != 0) {
- // not a power of two
- throw new IllegalArgumentException("The given page size is not a power of two.");
- }
-
- this.memorySize = memorySize;
-
- this.numberOfSlots = numberOfSlots;
-
- // assign page size and bit utilities
- this.pageSize = pageSize;
- this.roundingMask = ~((long) (pageSize - 1));
- int log = 0;
- while ((pageSize = pageSize >>> 1) != 0) {
- log++;
- }
- this.pageSizeBits = log;
-
- this.totalNumPages = getNumPages(memorySize);
- if (this.totalNumPages < 1) {
- throw new IllegalArgumentException("The given amount of memory amounted to less than one page.");
- }
-
- // initialize the free segments and allocated segments tracking structures
- this.freeSegments = new ArrayDeque<byte[]>(this.totalNumPages);
- this.allocatedSegments = new HashMap<AbstractInvokable, Set<DefaultMemorySegment>>();
-
- this.isPreAllocated = preAllocateMemory;
-
- if (preAllocateMemory) {
- // add the full chunks
- for (int i = 0; i < this.totalNumPages; i++) {
- // allocate memory of the specified size
- this.freeSegments.add(new byte[this.pageSize]);
- }
- }
- else {
- this.numNonAllocatedPages = this.totalNumPages;
- }
- }
-
- @Override
- public void shutdown() {
- // -------------------- BEGIN CRITICAL SECTION -------------------
- synchronized (this.lock)
- {
- if (!this.isShutDown) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Shutting down MemoryManager instance " + this);
- }
-
- // mark as shutdown and release memory
- this.isShutDown = true;
-
- this.freeSegments.clear();
- this.numNonAllocatedPages = 0;
-
- // go over all allocated segments and release them
- for (Set<DefaultMemorySegment> segments : this.allocatedSegments.values()) {
- for (DefaultMemorySegment seg : segments) {
- seg.destroy();
- }
- }
- }
- }
- // -------------------- END CRITICAL SECTION -------------------
- }
-
- @Override
- public boolean isShutdown() {
- return this.isShutDown;
- }
-
- @Override
- public boolean verifyEmpty() {
- synchronized (this.lock) {
- return isPreAllocated ?
- this.freeSegments.size() == this.totalNumPages :
- this.numNonAllocatedPages == this.totalNumPages;
- }
- }
-
- // ------------------------------------------------------------------------
- // MemoryManager interface implementation
- // ------------------------------------------------------------------------
-
- @Override
- public List<MemorySegment> allocatePages(AbstractInvokable owner, int numPages) throws MemoryAllocationException {
- final ArrayList<MemorySegment> segs = new ArrayList<MemorySegment>(numPages);
- allocatePages(owner, segs, numPages);
- return segs;
- }
-
- @Override
- public void allocatePages(AbstractInvokable owner, List<MemorySegment> target, int numPages)
- throws MemoryAllocationException
- {
- // sanity check
- if (owner == null) {
- throw new IllegalAccessError("The memory owner must not be null.");
- }
-
- // reserve array space, if applicable
- if (target instanceof ArrayList) {
- ((ArrayList<MemorySegment>) target).ensureCapacity(numPages);
- }
-
- // -------------------- BEGIN CRITICAL SECTION -------------------
- synchronized (this.lock)
- {
- if (this.isShutDown) {
- throw new IllegalStateException("Memory manager has been shut down.");
- }
-
- // in the case of pre-allocated memory, the 'numNonAllocatedPages' is zero, in the
- // lazy case, the 'freeSegments.size()' is zero.
- if (numPages > (this.freeSegments.size() + numNonAllocatedPages)) {
- throw new MemoryAllocationException("Could not allocate " + numPages + " pages. Only " +
- this.freeSegments.size() + " pages are remaining.");
- }
-
- Set<DefaultMemorySegment> segmentsForOwner = this.allocatedSegments.get(owner);
- if (segmentsForOwner == null) {
- segmentsForOwner = new HashSet<DefaultMemorySegment>(4 * numPages / 3 + 1);
- this.allocatedSegments.put(owner, segmentsForOwner);
- }
-
- if (isPreAllocated) {
- for (int i = numPages; i > 0; i--) {
- byte[] buffer = this.freeSegments.poll();
- final DefaultMemorySegment segment = new DefaultMemorySegment(owner, buffer);
- target.add(segment);
- segmentsForOwner.add(segment);
- }
- }
- else {
- for (int i = numPages; i > 0; i--) {
- byte[] buffer = new byte[pageSize];
- final DefaultMemorySegment segment = new DefaultMemorySegment(owner, buffer);
- target.add(segment);
- segmentsForOwner.add(segment);
- }
- numNonAllocatedPages -= numPages;
- }
- }
- // -------------------- END CRITICAL SECTION -------------------
- }
-
- // ------------------------------------------------------------------------
-
- @Override
- public void release(MemorySegment segment) {
- // check if segment is null or has already been freed
- if (segment == null || segment.isFreed() || !(segment instanceof DefaultMemorySegment)) {
- return;
- }
-
- final DefaultMemorySegment defSeg = (DefaultMemorySegment) segment;
- final AbstractInvokable owner = defSeg.owner;
-
- // -------------------- BEGIN CRITICAL SECTION -------------------
- synchronized (this.lock)
- {
- if (this.isShutDown) {
- throw new IllegalStateException("Memory manager has been shut down.");
- }
-
- // remove the reference in the map for the owner
- try {
- Set<DefaultMemorySegment> segsForOwner = this.allocatedSegments.get(owner);
-
- if (segsForOwner != null) {
- segsForOwner.remove(defSeg);
- if (segsForOwner.isEmpty()) {
- this.allocatedSegments.remove(owner);
- }
- }
-
- byte[] buffer = defSeg.destroy();
-
- if (isPreAllocated) {
- // release the memory in any case
- this.freeSegments.add(buffer);
- }
- else {
- numNonAllocatedPages++;
- }
- }
- catch (Throwable t) {
- throw new RuntimeException("Error removing book-keeping reference to allocated memory segment.", t);
- }
- }
- // -------------------- END CRITICAL SECTION -------------------
- }
-
- @Override
- public <T extends MemorySegment> void release(Collection<T> segments) {
- if (segments == null) {
- return;
- }
-
- // -------------------- BEGIN CRITICAL SECTION -------------------
- synchronized (this.lock)
- {
- if (this.isShutDown) {
- throw new IllegalStateException("Memory manager has been shut down.");
- }
-
- // since concurrent modifications to the collection
- // can disturb the release, we need to try potentially multiple times
- boolean successfullyReleased = false;
- do {
- final Iterator<T> segmentsIterator = segments.iterator();
-
- AbstractInvokable lastOwner = null;
- Set<DefaultMemorySegment> segsForOwner = null;
-
- try {
- // go over all segments
- while (segmentsIterator.hasNext()) {
-
- final MemorySegment seg = segmentsIterator.next();
- if (seg == null || seg.isFreed()) {
- continue;
- }
-
- final DefaultMemorySegment defSeg = (DefaultMemorySegment) seg;
- final AbstractInvokable owner = defSeg.owner;
-
- try {
- // get the list of segments by this owner only if it is a different owner than for
- // the previous one (or it is the first segment)
- if (lastOwner != owner) {
- lastOwner = owner;
- segsForOwner = this.allocatedSegments.get(owner);
- }
-
- // remove the segment from the list
- if (segsForOwner != null) {
- segsForOwner.remove(defSeg);
- if (segsForOwner.isEmpty()) {
- this.allocatedSegments.remove(owner);
- }
- }
-
- // release the memory in any case
- byte[] buffer = defSeg.destroy();
-
- if (isPreAllocated) {
- this.freeSegments.add(buffer);
- }
- else {
- numNonAllocatedPages++;
- }
- }
- catch (Throwable t) {
- throw new RuntimeException(
- "Error removing book-keeping reference to allocated memory segment.", t);
- }
- }
-
- segments.clear();
-
- // the only way to exit the loop
- successfullyReleased = true;
- }
- catch (ConcurrentModificationException e) {
- // this may happen in the case where an asynchronous
- // call releases the memory. fall through the loop and try again
- }
- } while (!successfullyReleased);
- }
- // -------------------- END CRITICAL SECTION -------------------
- }
-
- @Override
- public void releaseAll(AbstractInvokable owner) {
- // -------------------- BEGIN CRITICAL SECTION -------------------
- synchronized (this.lock)
- {
- if (this.isShutDown) {
- throw new IllegalStateException("Memory manager has been shut down.");
- }
-
- // get all segments
- final Set<DefaultMemorySegment> segments = this.allocatedSegments.remove(owner);
-
- // all segments may have been freed previously individually
- if (segments == null || segments.isEmpty()) {
- return;
- }
-
- // free each segment
- if (isPreAllocated) {
- for (DefaultMemorySegment seg : segments) {
- final byte[] buffer = seg.destroy();
- this.freeSegments.add(buffer);
- }
- }
- else {
- for (DefaultMemorySegment seg : segments) {
- seg.destroy();
- }
- numNonAllocatedPages += segments.size();
- }
-
- segments.clear();
- }
- // -------------------- END CRITICAL SECTION -------------------
- }
-
- // ------------------------------------------------------------------------
-
- @Override
- public int getPageSize() {
- return this.pageSize;
- }
-
- @Override
- public long getMemorySize() {
- return this.memorySize;
- }
-
- @Override
- public int computeNumberOfPages(double fraction) {
- if (fraction <= 0 || fraction > 1) {
- throw new IllegalArgumentException("The fraction of memory to allocate must within (0, 1].");
- }
-
- return (int)(this.totalNumPages * fraction / this.numberOfSlots);
- }
-
- @Override
- public long computeMemorySize(double fraction) {
- return this.pageSize * computeNumberOfPages(fraction);
- }
-
- @Override
- public long roundDownToPageSizeMultiple(long numBytes) {
- return numBytes & this.roundingMask;
- }
-
- // ------------------------------------------------------------------------
-
- private int getNumPages(long numBytes) {
- if (numBytes < 0) {
- throw new IllegalArgumentException("The number of bytes to allocate must not be negative.");
- }
-
- final long numPages = numBytes >>> this.pageSizeBits;
- if (numPages <= Integer.MAX_VALUE) {
- return (int) numPages;
- } else {
- throw new IllegalArgumentException("The given number of bytes corresponds to more than MAX_INT pages.");
- }
- }
-
- // ------------------------------------------------------------------------
-
- private static final class DefaultMemorySegment extends MemorySegment {
-
- private AbstractInvokable owner;
-
- DefaultMemorySegment(AbstractInvokable owner, byte[] memory) {
- super(memory);
- this.owner = owner;
- }
-
- byte[] destroy() {
- final byte[] buffer = this.memory;
- this.memory = null;
- this.wrapper = null;
- return buffer;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/ListMemorySegmentSource.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/ListMemorySegmentSource.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/ListMemorySegmentSource.java
deleted file mode 100644
index cacb6b0..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/ListMemorySegmentSource.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.runtime.memorymanager;
-
-import java.util.List;
-
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.core.memory.MemorySegmentSource;
-
-/**
- * Simple memory segment source that draws segments from a list.
- *
- */
-public class ListMemorySegmentSource implements MemorySegmentSource
-{
- private final List<MemorySegment> segments;
-
- public ListMemorySegmentSource(final List<MemorySegment> memorySegments) {
- this.segments = memorySegments;
- }
-
-
- @Override
- public MemorySegment nextSegment() {
- if (this.segments.size() > 0) {
- return this.segments.remove(this.segments.size() - 1);
- } else {
- return null;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryAllocationException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryAllocationException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryAllocationException.java
deleted file mode 100644
index d4d8b3b..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryAllocationException.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.runtime.memorymanager;
-
-/**
- * An exception to be thrown when a memory allocation operation is not successful.
- */
-public class MemoryAllocationException extends Exception {
-
- private static final long serialVersionUID = -403983866457947012L;
-
- public MemoryAllocationException() {
- super();
- }
-
- public MemoryAllocationException(String message, Throwable cause) {
- super(message, cause);
- }
-
- public MemoryAllocationException(String message) {
- super(message);
- }
-
- public MemoryAllocationException(Throwable cause) {
- super(cause);
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryManager.java
deleted file mode 100644
index 631f0b8..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryManager.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.runtime.memorymanager;
-
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-
-/**
- * Interface for a memory manager that assigns portions of memory to different tasks. Each allocated segment of
- * memory is specific to a task. Memory segments can be freed individually, or all memory allocated by a task can
- * be freed as a whole.
- * <p>
- * Internally, memory is represented as byte arrays. The memory manager acts like a distributer for memory, which
- * means it assigns portions of the arrays to tasks. If memory is released, it means that this part of the memory can
- * be assigned to other tasks.
- */
-public interface MemoryManager {
-
- List<MemorySegment> allocatePages(AbstractInvokable owner, int numPages) throws MemoryAllocationException;
-
- void allocatePages(AbstractInvokable owner, List<MemorySegment> target, int numPages) throws MemoryAllocationException;
-
- /**
- * Tries to release the memory for the specified segment. If the <code>segment</code> has already been released or
- * is <code>null</code>, the request is simply ignored. If the segment is not from the expected
- * MemorySegment implementation type, an <code>IllegalArgumentException</code> is thrown.
- *
- * @param segment The segment to be released.
- * @throws IllegalArgumentException Thrown, if the given segment is of an incompatible type.
- */
- void release(MemorySegment segment);
-
- /**
- * Tries to release the memory for the specified collection of segments.
- *
- * @param <T> The type of memory segment.
- * @param segments The segments to be released.
- * @throws NullPointerException Thrown, if the given collection is null.
- * @throws IllegalArgumentException Thrown, id the segments are of an incompatible type.
- */
- <T extends MemorySegment> void release(Collection<T> segments);
-
- /**
- * Releases all memory segments for the given task.
- *
- * @param task The task whose memory segments are to be released.
- */
- void releaseAll(AbstractInvokable task);
-
- // --------------------------------------------------------------------------------------------
-
- /**
- * Gets the size of the pages handled by the memory manager.
- *
- * @return The size of the pages handled by the memory manager.
- */
- int getPageSize();
-
- /**
- * Returns the total size of memory.
- *
- * @return The total size of memory.
- */
- long getMemorySize();
-
- /**
- * Computes to how many pages the given number of bytes corresponds. If the given number of bytes is not an
- * exact multiple of a page size, the result is rounded down, such that a portion of the memory (smaller
- * than the page size) is not included.
- *
- * @param fraction the fraction of the total memory per slot
- * @return The number of pages to which
- */
- int computeNumberOfPages(double fraction);
-
- /**
- * Computes the memory size of the fraction per slot.
- *
- * @param fraction The fraction of the memory of the task slot.
- * @return The number of pages corresponding to the memory fraction.
- */
- long computeMemorySize(double fraction);
-
- /**
- * Rounds the given value down to a multiple of the memory manager's page size.
- *
- * @return The given value, rounded down to a multiple of the page size.
- */
- long roundDownToPageSizeMultiple(long numBytes);
-
- // --------------------------------------------------------------------------------------------
-
- /**
- * Shuts the memory manager down, trying to release all the memory it managed. Depending
- * on implementation details, the memory does not necessarily become reclaimable by the
- * garbage collector, because there might still be references to allocated segments in the
- * code that allocated them from the memory manager.
- */
- void shutdown();
-
- /**
- * Checks whether the MemoryManager has been shut down.
- *
- * @return True, if the memory manager is shut down, false otherwise.
- */
- boolean isShutdown();
-
- /**
- * Checks if the memory manager all memory available.
- *
- * @return True, if the memory manager is empty and valid, false if it is not empty or corrupted.
- */
- boolean verifyEmpty();
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/SimpleMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/SimpleMemorySegment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/SimpleMemorySegment.java
deleted file mode 100644
index 7219503..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/SimpleMemorySegment.java
+++ /dev/null
@@ -1,329 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.runtime.memorymanager;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-/**
- * This class represents a piece of memory allocated from the memory manager. The segment is backed
- * by a byte array and features random put and get methods for the basic types that are stored in a byte-wise
- * fashion in the memory.
- */
-public class SimpleMemorySegment {
-
- /**
- * The array in which the data is stored.
- */
- protected byte[] memory;
-
- /**
- * Wrapper for I/O requests.
- */
- protected ByteBuffer wrapper;
-
- // -------------------------------------------------------------------------
- // Constructors
- // -------------------------------------------------------------------------
-
- /**
- * Creates a new memory segment of given size with the provided views.
- */
- public SimpleMemorySegment(byte[] memory) {
- this.memory = memory;
- }
-
- // -------------------------------------------------------------------------
- // MemorySegment Accessors
- // -------------------------------------------------------------------------
-
- /**
- * Checks whether this memory segment has already been freed. In that case, the
- * segment must not be used any more.
- *
- * @return True, if the segment has been freed, false otherwise.
- */
- public final boolean isFreed() {
- return this.memory == null;
- }
-
- /**
- * Gets the size of the memory segment, in bytes. Because segments
- * are backed by arrays, they cannot be larger than two GiBytes.
- *
- * @return The size in bytes.
- */
- public final int size() {
- return this.memory.length;
- }
-
- /**
- * Gets the byte array that backs the memory segment and this random access view.
- * Since different regions of the backing array are used by different segments, the logical
- * positions in this view do not correspond to the indexes in the backing array and need
- * to be translated via the {@link #translateOffset(int)} method.
- *
- * @return The backing byte array.
- */
- public final byte[] getBackingArray() {
- return this.memory;
- }
-
- /**
- * Translates the given offset for this view into the offset for the backing array.
- *
- * @param offset The offset to be translated.
- * @return The corresponding position in the backing array.
- */
- public final int translateOffset(int offset) {
- return offset;
- }
-
- // -------------------------------------------------------------------------
- // Helper methods
- // -------------------------------------------------------------------------
-
-
- /**
- * Wraps the chunk of the underlying memory located between <tt>offset<tt> and
- * <tt>length</tt> in a NIO ByteBuffer.
- *
- * @param offset The offset in the memory segment.
- * @param length The number of bytes to be wrapped as a buffer.
- * @return A <tt>ByteBuffer</tt> backed by the specified portion of the memory segment.
- * @throws IndexOutOfBoundsException Thrown, if offset is negative or larger than the memory segment size,
- * or if the offset plus the length is larger than the segment size.
- */
- public ByteBuffer wrap(int offset, int length) {
- if (offset > this.memory.length || offset > this.memory.length - length) {
- throw new IndexOutOfBoundsException();
- }
-
- if (this.wrapper == null) {
- this.wrapper = ByteBuffer.wrap(this.memory, offset, length);
- }
- else {
- this.wrapper.position(offset);
- this.wrapper.limit(offset + length);
- }
-
- return this.wrapper;
- }
-
-
- // --------------------------------------------------------------------
- // Random Access
- // --------------------------------------------------------------------
-
- // ------------------------------------------------------------------------------------------------------
- // WARNING: Any code for range checking must take care to avoid integer overflows. The position
- // integer may go up to <code>Integer.MAX_VALUE</tt>. Range checks that work after the principle
- // <code>position + 3 < end</code> may fail because <code>position + 3</code> becomes negative.
- // A safe solution is to subtract the delta from the limit, for example
- // <code>position < end - 3</code>. Since all indices are always positive, and the integer domain
- // has one more negative value than positive values, this can never cause an underflow.
- // ------------------------------------------------------------------------------------------------------
-
- public final byte get(int index) {
- return this.memory[index];
- }
-
- public final void put(int index, byte b) {
- this.memory[index] = b;
- }
-
- public final void get(int index, byte[] dst) {
- get(index, dst, 0, dst.length);
- }
-
- public final void put(int index, byte[] src) {
- put(index, src, 0, src.length);
- }
-
- public final void get(int index, byte[] dst, int offset, int length) {
- System.arraycopy(this.memory, index, dst, offset, length);
- }
-
- public final void put(int index, byte[] src, int offset, int length) {
- System.arraycopy(src, offset, this.memory, index, length);
- }
-
- public final void get(DataOutput out, int offset, int length) throws IOException {
- out.write(this.memory, offset, length);
- }
-
- public final void put(DataInput in, int offset, int length) throws IOException {
- in.readFully(this.memory, offset, length);
- }
-
- public final boolean getBoolean(int index) {
- return this.memory[index] != 0;
- }
-
- public final void putBoolean(int index, boolean value) {
- this.memory[index] = (byte) (value ? 1 : 0);
- }
-
- public final char getChar(int index) {
- return (char) ( ((this.memory[index ] & 0xff) << 8) |
- (this.memory[index + 1] & 0xff) );
- }
-
- public final void putChar(int index, char value) {
- this.memory[index ] = (byte) (value >> 8);
- this.memory[index + 1] = (byte) value;
- }
-
- public final short getShort(int index) {
- return (short) (
- ((this.memory[index ] & 0xff) << 8) |
- ((this.memory[index + 1] & 0xff)) );
- }
-
- public final void putShort(int index, short value) {
- this.memory[index ] = (byte) (value >> 8);
- this.memory[index + 1] = (byte) value;
- }
-
- public final int getInt(int index) {
- return ((this.memory[index ] & 0xff) << 24)
- | ((this.memory[index + 1] & 0xff) << 16)
- | ((this.memory[index + 2] & 0xff) << 8)
- | ((this.memory[index + 3] & 0xff) );
- }
-
- public final int getIntLittleEndian(int index) {
- return ((this.memory[index ] & 0xff) )
- | ((this.memory[index + 1] & 0xff) << 8)
- | ((this.memory[index + 2] & 0xff) << 16)
- | ((this.memory[index + 3] & 0xff) << 24);
- }
-
- public final int getIntBigEndian(int index) {
- return ((this.memory[index ] & 0xff) << 24)
- | ((this.memory[index + 1] & 0xff) << 16)
- | ((this.memory[index + 2] & 0xff) << 8)
- | ((this.memory[index + 3] & 0xff) );
- }
-
- public final void putInt(int index, int value) {
- this.memory[index ] = (byte) (value >> 24);
- this.memory[index + 1] = (byte) (value >> 16);
- this.memory[index + 2] = (byte) (value >> 8);
- this.memory[index + 3] = (byte) value;
- }
-
- public final void putIntLittleEndian(int index, int value) {
- this.memory[index ] = (byte) value;
- this.memory[index + 1] = (byte) (value >> 8);
- this.memory[index + 2] = (byte) (value >> 16);
- this.memory[index + 3] = (byte) (value >> 24);
- }
-
- public final void putIntBigEndian(int index, int value) {
- this.memory[index ] = (byte) (value >> 24);
- this.memory[index + 1] = (byte) (value >> 16);
- this.memory[index + 2] = (byte) (value >> 8);
- this.memory[index + 3] = (byte) value;
- }
-
- public final long getLong(int index) {
- return (((long) this.memory[index ] & 0xff) << 56)
- | (((long) this.memory[index + 1] & 0xff) << 48)
- | (((long) this.memory[index + 2] & 0xff) << 40)
- | (((long) this.memory[index + 3] & 0xff) << 32)
- | (((long) this.memory[index + 4] & 0xff) << 24)
- | (((long) this.memory[index + 5] & 0xff) << 16)
- | (((long) this.memory[index + 6] & 0xff) << 8)
- | (((long) this.memory[index + 7] & 0xff) );
- }
- public final long getLongLittleEndian(int index) {
- return (((long) this.memory[index ] & 0xff) )
- | (((long) this.memory[index + 1] & 0xff) << 8)
- | (((long) this.memory[index + 2] & 0xff) << 16)
- | (((long) this.memory[index + 3] & 0xff) << 24)
- | (((long) this.memory[index + 4] & 0xff) << 32)
- | (((long) this.memory[index + 5] & 0xff) << 40)
- | (((long) this.memory[index + 6] & 0xff) << 48)
- | (((long) this.memory[index + 7] & 0xff) << 56);
- }
-
- public final long getLongBigEndian(int index) {
- return (((long) this.memory[index ] & 0xff) << 56)
- | (((long) this.memory[index + 1] & 0xff) << 48)
- | (((long) this.memory[index + 2] & 0xff) << 40)
- | (((long) this.memory[index + 3] & 0xff) << 32)
- | (((long) this.memory[index + 4] & 0xff) << 24)
- | (((long) this.memory[index + 5] & 0xff) << 16)
- | (((long) this.memory[index + 6] & 0xff) << 8)
- | (((long) this.memory[index + 7] & 0xff) );
- }
-
- public final void putLong(int index, long value) {
- this.memory[index ] = (byte) (value >> 56);
- this.memory[index + 1] = (byte) (value >> 48);
- this.memory[index + 2] = (byte) (value >> 40);
- this.memory[index + 3] = (byte) (value >> 32);
- this.memory[index + 4] = (byte) (value >> 24);
- this.memory[index + 5] = (byte) (value >> 16);
- this.memory[index + 6] = (byte) (value >> 8);
- this.memory[index + 7] = (byte) value;
- }
-
- public final void putLongLittleEndian(int index, long value) {
- this.memory[index ] = (byte) value;
- this.memory[index + 1] = (byte) (value >> 8);
- this.memory[index + 2] = (byte) (value >> 16);
- this.memory[index + 3] = (byte) (value >> 24);
- this.memory[index + 4] = (byte) (value >> 32);
- this.memory[index + 5] = (byte) (value >> 40);
- this.memory[index + 6] = (byte) (value >> 48);
- this.memory[index + 7] = (byte) (value >> 56);
- }
-
- public final void putLongBigEndian(int index, long value) {
- this.memory[index ] = (byte) (value >> 56);
- this.memory[index + 1] = (byte) (value >> 48);
- this.memory[index + 2] = (byte) (value >> 40);
- this.memory[index + 3] = (byte) (value >> 32);
- this.memory[index + 4] = (byte) (value >> 24);
- this.memory[index + 5] = (byte) (value >> 16);
- this.memory[index + 6] = (byte) (value >> 8);
- this.memory[index + 7] = (byte) value;
- }
-
- public final float getFloat(int index) {
- return Float.intBitsToFloat(getInt(index));
- }
-
- public final void putFloat(int index, float value) {
- putLong(index, Float.floatToIntBits(value));
- }
-
- public final double getDouble(int index) {
- return Double.longBitsToDouble(getLong(index));
- }
-
- public final void putDouble(int index, double value) {
- putLong(index, Double.doubleToLongBits(value));
- }
-}
[12/15] flink git commit: [FLINK-1320] [core] Add an off-heap variant
of the managed memory
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java
new file mode 100644
index 0000000..7b8d996
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java
@@ -0,0 +1,1367 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory;
+
+import org.junit.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class MemorySegmentUndersizedTest {
+
+ @Test
+ public void testZeroSizeHeapSegment() {
+ try {
+ MemorySegment segment = new HeapMemorySegment(new byte[0]);
+
+ testZeroSizeBuffer(segment);
+ testSegmentWithSizeLargerZero(segment);
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testZeroSizeHeapHybridSegment() {
+ try {
+ MemorySegment segment = new HybridMemorySegment(new byte[0]);
+
+ testZeroSizeBuffer(segment);
+ testSegmentWithSizeLargerZero(segment);
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testZeroSizeOffHeapHybridSegment() {
+ try {
+ MemorySegment segment = new HybridMemorySegment(ByteBuffer.allocateDirect(0));
+
+ testZeroSizeBuffer(segment);
+ testSegmentWithSizeLargerZero(segment);
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testSizeOneHeapSegment() {
+ try {
+ testSegmentWithSizeLargerZero(new HeapMemorySegment(new byte[1]));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testSizeOneHeapHybridSegment() {
+ try {
+ testSegmentWithSizeLargerZero(new HybridMemorySegment(new byte[1]));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testSizeOneOffHeapHybridSegment() {
+ try {
+ testSegmentWithSizeLargerZero(new HybridMemorySegment(ByteBuffer.allocateDirect(1)));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+
+ private static void testZeroSizeBuffer(MemorySegment segment) {
+ // ------ bytes ------
+
+ try {
+ segment.put(0, (byte) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // ------ booleans ------
+
+ try {
+ segment.putBoolean(0, true);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getBoolean(0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ }
+
+ private static void testSegmentWithSizeLargerZero(MemorySegment segment) {
+
+ // ------ bytes ------
+
+ try {
+ segment.put(1, (byte) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(-1, (byte) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(8, (byte) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(-8, (byte) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(Integer.MAX_VALUE, (byte) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(Integer.MIN_VALUE, (byte) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(-8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // ------ booleans ------
+
+ try {
+ segment.putBoolean(1, true);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putBoolean(-1, true);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putBoolean(8, true);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putBoolean(-8, true);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putBoolean(Integer.MAX_VALUE, true);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putBoolean(Integer.MIN_VALUE, true);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getBoolean(1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getBoolean(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getBoolean(8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getBoolean(-8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getBoolean(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getBoolean(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // ------ chars ------
+
+ try {
+ segment.putChar(0, 'a');
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putChar(1, 'a');
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putChar(-1, 'a');
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putChar(8, 'a');
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putChar(-8, 'a');
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putChar(Integer.MAX_VALUE, 'a');
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putChar(Integer.MIN_VALUE, 'a');
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getChar(0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getChar(1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getChar(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getChar(8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getChar(-8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getChar(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getChar(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // ------ shorts ------
+
+ try {
+ segment.putShort(0, (short) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putShort(1, (short) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putShort(-1, (short) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putShort(8, (short) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putShort(-8, (short) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putShort(Integer.MAX_VALUE, (short) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putShort(Integer.MIN_VALUE, (short) 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getShort(0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getShort(1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getShort(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getShort(8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getShort(-8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getShort(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getShort(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // ------ ints ------
+
+ try {
+ segment.putInt(0, 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putInt(1, 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putInt(-1, 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putInt(8, 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putInt(-8, 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putInt(Integer.MAX_VALUE, 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putInt(Integer.MIN_VALUE, 0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getInt(0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ try {
+ segment.getInt(1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getInt(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getInt(8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getInt(-8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getInt(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getInt(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // ------ longs ------
+
+ try {
+ segment.putLong(0, 0L);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putLong(1, 0L);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putLong(-1, 0L);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putLong(8, 0L);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putLong(-8, 0L);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putLong(Integer.MAX_VALUE, 0L);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putLong(Integer.MIN_VALUE, 0L);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getLong(0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ try {
+ segment.getLong(1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getLong(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getLong(8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getLong(-8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getLong(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getLong(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // ------ floats ------
+
+ try {
+ segment.putFloat(0, 0.0f);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putFloat(1, 0.0f);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putFloat(-1, 0.0f);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putFloat(8, 0.0f);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putFloat(-8, 0.0f);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putFloat(Integer.MAX_VALUE, 0.0f);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putFloat(Integer.MIN_VALUE, 0.0f);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getFloat(0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ try {
+ segment.getFloat(1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getFloat(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getFloat(8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getFloat(-8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getFloat(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getFloat(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // ------ doubles ------
+
+ try {
+ segment.putDouble(0, 0.0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putDouble(1, 0.0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putDouble(-1, 0.0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putDouble(8, 0.0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putDouble(Integer.MAX_VALUE, 0.0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.putDouble(Integer.MIN_VALUE, 0.0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getDouble(0);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ try {
+ segment.getDouble(1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getDouble(-1);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getDouble(8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getDouble(-8);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getDouble(Integer.MAX_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.getDouble(Integer.MIN_VALUE);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+
+ // ------ byte[] ------
+
+ try {
+ segment.put(0, new byte[7]);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(1, new byte[7]);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(-1, new byte[7]);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(8, new byte[7]);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(-8, new byte[7]);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(Integer.MAX_VALUE, new byte[7]);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(Integer.MIN_VALUE, new byte[7]);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(0, new byte[7]);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(1, new byte[7]);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(-1, new byte[7]);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(8, new byte[7]);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(-8, new byte[7]);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(Integer.MAX_VALUE, new byte[7]);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(Integer.MIN_VALUE, new byte[7]);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // ------ ByteBuffer ------
+
+ final ByteBuffer buf = ByteBuffer.allocate(7);
+ final int numBytes = 3;
+
+ try {
+ segment.put(0, buf, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(1, buf, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(-1, buf, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(8, buf, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(-8, buf, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(Integer.MAX_VALUE, buf, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(Integer.MIN_VALUE, buf, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(0, buf, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ try {
+ segment.get(1, buf, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(-1, buf, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(8, buf, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(-8, buf, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(Integer.MAX_VALUE, buf, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(Integer.MIN_VALUE, buf, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ // ------ DataInput / DataOutput ------
+
+ final DataInput dataInput = new DataInputStream(new ByteArrayInputStream(new byte[20]));
+ final DataOutput dataOutput = new DataOutputStream(new ByteArrayOutputStream());
+
+ try {
+ segment.put(dataInput, 0, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(dataInput, 1, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(dataInput, -1, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(dataInput, 8, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(dataInput, -8, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(dataInput, Integer.MAX_VALUE, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.put(dataInput, Integer.MIN_VALUE, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(dataOutput, 0, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ try {
+ segment.get(dataOutput, 1, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(dataOutput, -1, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(dataOutput, 8, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(dataOutput, -8, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(dataOutput, Integer.MAX_VALUE, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+
+ try {
+ segment.get(dataOutput, Integer.MIN_VALUE, numBytes);
+ fail("IndexOutOfBoundsException expected");
+ }
+ catch (Exception e) {
+ assertTrue(e instanceof IndexOutOfBoundsException);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java
new file mode 100644
index 0000000..ddbdf72
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java
@@ -0,0 +1,1195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory;
+
+import org.junit.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.*;
+
+public class OperationsOnFreedSegmentTest {
+
+ private static final int PAGE_SIZE = (int) ((Math.random() * 10000) + 1000);
+
+ @Test
+ public void testSingleSegmentOperationsHeapSegment() {
+ try {
+ testOpsOnFreedSegment(new HeapMemorySegment(new byte[PAGE_SIZE]));
+ testOpsOnFreedSegment(new HybridMemorySegment(new byte[PAGE_SIZE]));
+ testOpsOnFreedSegment(new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE)));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testCompare() {
+ try {
+ MemorySegment aliveHeap = new HeapMemorySegment(new byte[PAGE_SIZE]);
+ MemorySegment aliveHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]);
+ MemorySegment aliveHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE));
+
+ MemorySegment freedHeap = new HeapMemorySegment(new byte[PAGE_SIZE]);
+ MemorySegment freedHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]);
+ MemorySegment freedHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE));
+ freedHeap.free();
+ freedHybridHeap.free();
+ freedHybridOffHeap.free();
+
+ MemorySegment[] alive = { aliveHeap, aliveHybridHeap, aliveHybridOffHeap };
+ MemorySegment[] free = { freedHeap, freedHybridHeap, freedHybridOffHeap };
+
+ // alive with free
+ for (MemorySegment seg1 : alive) {
+ for (MemorySegment seg2 : free) {
+ testCompare(seg1, seg2);
+ }
+ }
+
+ // free with alive
+ for (MemorySegment seg1 : free) {
+ for (MemorySegment seg2 : alive) {
+ testCompare(seg1, seg2);
+ }
+ }
+
+ // free with free
+ for (MemorySegment seg1 : free) {
+ for (MemorySegment seg2 : free) {
+ testCompare(seg1, seg2);
+ }
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testCopyTo() {
+ try {
+ MemorySegment aliveHeap = new HeapMemorySegment(new byte[PAGE_SIZE]);
+ MemorySegment aliveHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]);
+ MemorySegment aliveHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE));
+
+ MemorySegment freedHeap = new HeapMemorySegment(new byte[PAGE_SIZE]);
+ MemorySegment freedHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]);
+ MemorySegment freedHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE));
+ freedHeap.free();
+ freedHybridHeap.free();
+ freedHybridOffHeap.free();
+
+ MemorySegment[] alive = { aliveHeap, aliveHybridHeap, aliveHybridOffHeap };
+ MemorySegment[] free = { freedHeap, freedHybridHeap, freedHybridOffHeap };
+
+ // alive with free
+ for (MemorySegment seg1 : alive) {
+ for (MemorySegment seg2 : free) {
+ testCopy(seg1, seg2);
+ }
+ }
+
+ // free with alive
+ for (MemorySegment seg1 : free) {
+ for (MemorySegment seg2 : alive) {
+ testCopy(seg1, seg2);
+ }
+ }
+
+ // free with free
+ for (MemorySegment seg1 : free) {
+ for (MemorySegment seg2 : free) {
+ testCopy(seg1, seg2);
+ }
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testSwap() {
+ try {
+ MemorySegment aliveHeap = new HeapMemorySegment(new byte[PAGE_SIZE]);
+ MemorySegment aliveHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]);
+ MemorySegment aliveHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE));
+
+ MemorySegment freedHeap = new HeapMemorySegment(new byte[PAGE_SIZE]);
+ MemorySegment freedHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]);
+ MemorySegment freedHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE));
+ freedHeap.free();
+ freedHybridHeap.free();
+ freedHybridOffHeap.free();
+
+ MemorySegment[] alive = { aliveHeap, aliveHybridHeap, aliveHybridOffHeap };
+ MemorySegment[] free = { freedHeap, freedHybridHeap, freedHybridOffHeap };
+
+ // alive with free
+ for (MemorySegment seg1 : alive) {
+ for (MemorySegment seg2 : free) {
+ testSwap(seg1, seg2);
+ }
+ }
+
+ // free with alive
+ for (MemorySegment seg1 : free) {
+ for (MemorySegment seg2 : alive) {
+ testSwap(seg1, seg2);
+ }
+ }
+
+ // free with free
+ for (MemorySegment seg1 : free) {
+ for (MemorySegment seg2 : free) {
+ testSwap(seg1, seg2);
+ }
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ private void testOpsOnFreedSegment(MemorySegment segment) throws Exception {
+ segment.free();
+ assertTrue(segment.isFreed());
+
+ // --------- bytes -----------
+
+ try {
+ segment.get(0);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.get(-1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.get(1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.get(segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.get(-segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.get(Integer.MAX_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.get(Integer.MIN_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.put(0, (byte) 0);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.put(-1, (byte) 0);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.put(1, (byte) 0);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.put(segment.size(), (byte) 0);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.put(-segment.size(), (byte) 0);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.put(Integer.MAX_VALUE, (byte) 0);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.put(Integer.MIN_VALUE, (byte) 0);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ // --------- booleans -----------
+
+ try {
+ segment.getBoolean(0);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.getBoolean(-1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.getBoolean(1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.getBoolean(segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.getBoolean(-segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.getBoolean(Integer.MAX_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.getBoolean(Integer.MIN_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.putBoolean(0, true);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.putBoolean(-1, true);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.putBoolean(1, true);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.putBoolean(segment.size(), true);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.putBoolean(-segment.size(), true);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.putBoolean(Integer.MAX_VALUE, true);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.putBoolean(Integer.MIN_VALUE, true);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ // --------- char -----------
+
+ try {
+ segment.getChar(0);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getChar(-1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.getChar(1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getChar(segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getChar(-segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.getChar(Integer.MAX_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getChar(Integer.MIN_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putChar(0, 'a');
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putChar(-1, 'a');
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.putChar(1, 'a');
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putChar(segment.size(), 'a');
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putChar(-segment.size(), 'a');
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.putChar(Integer.MAX_VALUE, 'a');
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putChar(Integer.MIN_VALUE, 'a');
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ // --------- short -----------
+
+ try {
+ segment.getShort(0);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getShort(-1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.getShort(1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getShort(segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getShort(-segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.getShort(Integer.MAX_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getShort(Integer.MIN_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putShort(0, (short) 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putShort(-1, (short) 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.putShort(1, (short) 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putShort(segment.size(), (short) 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putShort(-segment.size(), (short) 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.putShort(Integer.MAX_VALUE, (short) 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putShort(Integer.MIN_VALUE, (short) 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ // --------- integer -----------
+
+ try {
+ segment.getInt(0);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getInt(-1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.getInt(1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getInt(segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getInt(-segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.getInt(Integer.MAX_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getInt(Integer.MIN_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putInt(0, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putInt(-1, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.putInt(1, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putInt(segment.size(), 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putInt(-segment.size(), 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.putInt(Integer.MAX_VALUE, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putInt(Integer.MIN_VALUE, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ // --------- longs -----------
+
+ try {
+ segment.getLong(0);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getLong(-1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.getLong(1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getLong(segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getLong(-segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.getLong(Integer.MAX_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getLong(Integer.MIN_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putLong(0, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putLong(-1, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.putLong(1, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putLong(segment.size(), 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putLong(-segment.size(), 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.putLong(Integer.MAX_VALUE, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putLong(Integer.MIN_VALUE, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ // --------- floats -----------
+
+ try {
+ segment.getFloat(0);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getFloat(-1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.getFloat(1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getFloat(segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getFloat(-segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.getFloat(Integer.MAX_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getFloat(Integer.MIN_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putFloat(0, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putFloat(-1, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.putFloat(1, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putFloat(segment.size(), 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putFloat(-segment.size(), 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.putFloat(Integer.MAX_VALUE, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putFloat(Integer.MIN_VALUE, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ // --------- doubles -----------
+
+ try {
+ segment.getDouble(0);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getDouble(-1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.getDouble(1);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getDouble(segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getDouble(-segment.size());
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.getDouble(Integer.MAX_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.getDouble(Integer.MIN_VALUE);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putDouble(0, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putDouble(-1, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.putDouble(1, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putDouble(segment.size(), 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putDouble(-segment.size(), 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.putDouble(Integer.MAX_VALUE, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ try {
+ segment.putDouble(Integer.MIN_VALUE, 42);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException ignored) {}
+
+ // --------- byte[] -----------
+
+ final byte[] array = new byte[55];
+
+ try {
+ segment.get(0, array, 3, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.get(-1, array, 3, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.get(1, array, 3, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.get(segment.size(), array, 3, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.get(-segment.size(), array, 3, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.get(Integer.MAX_VALUE, array, 3, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.get(Integer.MIN_VALUE, array, 3, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.put(0, array, 3, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.put(-1, array, 3, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.put(1, array, 3, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.put(segment.size(), array, 3, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.put(-segment.size(), array, 3, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.put(Integer.MAX_VALUE, array, 3, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.put(Integer.MIN_VALUE, array, 3, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ // --------- ByteBuffer -----------
+
+ for (ByteBuffer bbuf : new ByteBuffer[] {
+ ByteBuffer.allocate(55),
+ ByteBuffer.allocateDirect(55) } )
+ {
+ try {
+ segment.get(0, bbuf, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.get(-1, bbuf, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.get(1, bbuf, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.get(segment.size(), bbuf, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.get(-segment.size(), bbuf, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.get(Integer.MAX_VALUE, bbuf, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.get(Integer.MIN_VALUE, bbuf, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.put(0, bbuf, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.put(-1, bbuf, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.put(1, bbuf, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.put(segment.size(), bbuf, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.put(-segment.size(), bbuf, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.put(Integer.MAX_VALUE,bbuf, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.put(Integer.MIN_VALUE, bbuf, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+ }
+
+ // --------- Data Input / Output -----------
+
+ final DataInput din = new DataInputStream(new ByteArrayInputStream(new byte[100]));
+ final DataOutput dout = new DataOutputStream(new ByteArrayOutputStream());
+
+ try {
+ segment.get(dout, 0, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.get(dout, -1, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.get(dout, 1, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.get(dout, segment.size(), 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.get(dout, -segment.size(), 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.get(dout, Integer.MAX_VALUE, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.get(dout, Integer.MIN_VALUE, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.put(din, 0, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.put(din, -1, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.put(din, 1, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.put(din, segment.size(), 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException ignored) {}
+
+ try {
+ segment.put(din, -segment.size(), 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.put(din, Integer.MAX_VALUE, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+ try {
+ segment.put(din, Integer.MIN_VALUE, 17);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+ }
+
+
+ private void testCompare(MemorySegment seg1, MemorySegment seg2) {
+ int[] offsetsToTest = { 0, 1, -1, seg1.size(), -seg1.size(), Integer.MAX_VALUE, Integer.MIN_VALUE };
+ int[] lengthsToTest = { 1, seg1.size(), Integer.MAX_VALUE };
+
+ for (int off1 : offsetsToTest) {
+ for (int off2 : offsetsToTest) {
+ for (int len : lengthsToTest) {
+ try {
+ seg1.compare(seg2, off1, off2, len);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException | NullPointerException ignored) {}
+ }
+ }
+ }
+ }
+
+ private void testCopy(MemorySegment seg1, MemorySegment seg2) {
+ int[] offsetsToTest = { 0, 1, -1, seg1.size(), -seg1.size(), Integer.MAX_VALUE, Integer.MIN_VALUE };
+ int[] lengthsToTest = { 0, 1, -1, seg1.size(), -seg1.size(), Integer.MAX_VALUE, Integer.MIN_VALUE };
+
+ for (int off1 : offsetsToTest) {
+ for (int off2 : offsetsToTest) {
+ for (int len : lengthsToTest) {
+ try {
+ seg1.copyTo(off1, seg2, off2, len);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException | NullPointerException ignored) {}
+ }
+ }
+ }
+ }
+
+ private void testSwap(MemorySegment seg1, MemorySegment seg2) {
+ int[] offsetsToTest = { 0, 1, -1, seg1.size(), -seg1.size(), Integer.MAX_VALUE, Integer.MIN_VALUE };
+ int[] lengthsToTest = { 0, 1, -1, seg1.size(), -seg1.size(), Integer.MAX_VALUE, Integer.MIN_VALUE };
+ byte[] swapBuffer = new byte[seg1.size()];
+
+ for (int off1 : offsetsToTest) {
+ for (int off2 : offsetsToTest) {
+ for (int len : lengthsToTest) {
+ try {
+ seg1.swapBytes(swapBuffer, seg2, off1, off2, len);
+ fail("Should fail with an exception");
+ }
+ catch (IllegalStateException | IndexOutOfBoundsException | NullPointerException ignored) {}
+ }
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/CoreMemorySegmentOutView.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/CoreMemorySegmentOutView.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/CoreMemorySegmentOutView.java
new file mode 100644
index 0000000..df3f8be
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/CoreMemorySegmentOutView.java
@@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory.benchmarks;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.util.List;
+
+public final class CoreMemorySegmentOutView implements DataOutputView {
+
+ private MemorySegment currentSegment; // the current memory segment to write to
+
+ private int positionInSegment; // the offset in the current segment
+
+ private final int segmentSize; // the size of the memory segments
+
+ private final List<MemorySegment> memorySource;
+
+ private final List<MemorySegment> fullSegments;
+
+
+ private byte[] utfBuffer; // the reusable array for UTF encodings
+
+
+ public CoreMemorySegmentOutView(List<MemorySegment> emptySegments,
+ List<MemorySegment> fullSegmentTarget, int segmentSize) {
+ this.segmentSize = segmentSize;
+ this.currentSegment = emptySegments.remove(emptySegments.size() - 1);
+
+ this.memorySource = emptySegments;
+ this.fullSegments = fullSegmentTarget;
+ this.fullSegments.add(getCurrentSegment());
+ }
+
+
+ public void reset() {
+ if (this.fullSegments.size() != 0) {
+ throw new IllegalStateException("The target list still contains memory segments.");
+ }
+
+ clear();
+ try {
+ advance();
+ }
+ catch (IOException ioex) {
+ throw new RuntimeException("Error getting first segment for record collector.", ioex);
+ }
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Page Management
+ // --------------------------------------------------------------------------------------------
+
+ public MemorySegment nextSegment(MemorySegment current, int positionInCurrent) throws EOFException {
+ int size = this.memorySource.size();
+ if (size > 0) {
+ final MemorySegment next = this.memorySource.remove(size - 1);
+ this.fullSegments.add(next);
+ return next;
+ } else {
+ throw new EOFException();
+ }
+ }
+
+ public MemorySegment getCurrentSegment() {
+ return this.currentSegment;
+ }
+
+ public int getCurrentPositionInSegment() {
+ return this.positionInSegment;
+ }
+
+ public int getSegmentSize() {
+ return this.segmentSize;
+ }
+
+ protected void advance() throws IOException {
+ this.currentSegment = nextSegment(this.currentSegment, this.positionInSegment);
+ this.positionInSegment = 0;
+ }
+
+ protected void seekOutput(MemorySegment seg, int position) {
+ this.currentSegment = seg;
+ this.positionInSegment = position;
+ }
+
+ protected void clear() {
+ this.currentSegment = null;
+ this.positionInSegment = 0;
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Data Output Specific methods
+ // --------------------------------------------------------------------------------------------
+
+ @Override
+ public void write(int b) throws IOException {
+ writeByte(b);
+ }
+
+ @Override
+ public void write(byte[] b) throws IOException {
+ write(b, 0, b.length);
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ int remaining = this.segmentSize - this.positionInSegment;
+ if (remaining >= len) {
+ this.currentSegment.put(this.positionInSegment, b, off, len);
+ this.positionInSegment += len;
+ }
+ else {
+ if (remaining == 0) {
+ advance();
+ remaining = this.segmentSize - this.positionInSegment;
+ }
+ while (true) {
+ int toPut = Math.min(remaining, len);
+ this.currentSegment.put(this.positionInSegment, b, off, toPut);
+ off += toPut;
+ len -= toPut;
+
+ if (len > 0) {
+ this.positionInSegment = this.segmentSize;
+ advance();
+ remaining = this.segmentSize - this.positionInSegment;
+ }
+ else {
+ this.positionInSegment += toPut;
+ break;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void writeBoolean(boolean v) throws IOException {
+ writeByte(v ? 1 : 0);
+ }
+
+ @Override
+ public void writeByte(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize) {
+ this.currentSegment.put(this.positionInSegment++, (byte) v);
+ }
+ else {
+ advance();
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeShort(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 1) {
+ this.currentSegment.putShortBigEndian(this.positionInSegment, (short) v);
+ this.positionInSegment += 2;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeShort(v);
+ }
+ else {
+ writeByte(v >> 8);
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeChar(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 1) {
+ this.currentSegment.putCharBigEndian(this.positionInSegment, (char) v);
+ this.positionInSegment += 2;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeChar(v);
+ }
+ else {
+ writeByte(v >> 8);
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeInt(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 3) {
+ this.currentSegment.putIntBigEndian(this.positionInSegment, v);
+ this.positionInSegment += 4;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeInt(v);
+ }
+ else {
+ writeByte(v >> 24);
+ writeByte(v >> 16);
+ writeByte(v >> 8);
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeLong(long v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 7) {
+ this.currentSegment.putLongBigEndian(this.positionInSegment, v);
+ this.positionInSegment += 8;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeLong(v);
+ }
+ else {
+ writeByte((int) (v >> 56));
+ writeByte((int) (v >> 48));
+ writeByte((int) (v >> 40));
+ writeByte((int) (v >> 32));
+ writeByte((int) (v >> 24));
+ writeByte((int) (v >> 16));
+ writeByte((int) (v >> 8));
+ writeByte((int) v);
+ }
+ }
+
+ @Override
+ public void writeFloat(float v) throws IOException {
+ writeInt(Float.floatToRawIntBits(v));
+ }
+
+ @Override
+ public void writeDouble(double v) throws IOException {
+ writeLong(Double.doubleToRawLongBits(v));
+ }
+
+ @Override
+ public void writeBytes(String s) throws IOException {
+ for (int i = 0; i < s.length(); i++) {
+ writeByte(s.charAt(i));
+ }
+ }
+
+ @Override
+ public void writeChars(String s) throws IOException {
+ for (int i = 0; i < s.length(); i++) {
+ writeChar(s.charAt(i));
+ }
+ }
+
+ @Override
+ public void writeUTF(String str) throws IOException {
+ int strlen = str.length();
+ int utflen = 0;
+ int c, count = 0;
+
+ /* use charAt instead of copying String to char array */
+ for (int i = 0; i < strlen; i++) {
+ c = str.charAt(i);
+ if ((c >= 0x0001) && (c <= 0x007F)) {
+ utflen++;
+ } else if (c > 0x07FF) {
+ utflen += 3;
+ } else {
+ utflen += 2;
+ }
+ }
+
+ if (utflen > 65535) {
+ throw new UTFDataFormatException("encoded string too long: " + utflen + " memory");
+ }
+
+ if (this.utfBuffer == null || this.utfBuffer.length < utflen + 2) {
+ this.utfBuffer = new byte[utflen + 2];
+ }
+ final byte[] bytearr = this.utfBuffer;
+
+ bytearr[count++] = (byte) ((utflen >>> 8) & 0xFF);
+ bytearr[count++] = (byte) (utflen & 0xFF);
+
+ int i = 0;
+ for (i = 0; i < strlen; i++) {
+ c = str.charAt(i);
+ if (!((c >= 0x0001) && (c <= 0x007F))) {
+ break;
+ }
+ bytearr[count++] = (byte) c;
+ }
+
+ for (; i < strlen; i++) {
+ c = str.charAt(i);
+ if ((c >= 0x0001) && (c <= 0x007F)) {
+ bytearr[count++] = (byte) c;
+
+ } else if (c > 0x07FF) {
+ bytearr[count++] = (byte) (0xE0 | ((c >> 12) & 0x0F));
+ bytearr[count++] = (byte) (0x80 | ((c >> 6) & 0x3F));
+ bytearr[count++] = (byte) (0x80 | (c & 0x3F));
+ } else {
+ bytearr[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F));
+ bytearr[count++] = (byte) (0x80 | (c & 0x3F));
+ }
+ }
+
+ write(bytearr, 0, utflen + 2);
+ }
+
+ @Override
+ public void skipBytesToWrite(int numBytes) throws IOException {
+ while (numBytes > 0) {
+ final int remaining = this.segmentSize - this.positionInSegment;
+ if (numBytes <= remaining) {
+ this.positionInSegment += numBytes;
+ return;
+ }
+ this.positionInSegment = this.segmentSize;
+ advance();
+ numBytes -= remaining;
+ }
+ }
+
+ @Override
+ public void write(DataInputView source, int numBytes) throws IOException {
+ while (numBytes > 0) {
+ final int remaining = this.segmentSize - this.positionInSegment;
+ if (numBytes <= remaining) {
+ this.currentSegment.put(source, this.positionInSegment, numBytes);
+ this.positionInSegment += numBytes;
+ return;
+ }
+
+ if (remaining > 0) {
+ this.currentSegment.put(source, this.positionInSegment, remaining);
+ this.positionInSegment = this.segmentSize;
+ numBytes -= remaining;
+ }
+
+ advance();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/LongSerializationSpeedBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/LongSerializationSpeedBenchmark.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/LongSerializationSpeedBenchmark.java
new file mode 100644
index 0000000..ed3c1f5
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/LongSerializationSpeedBenchmark.java
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.core.memory.benchmarks;
+
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.core.memory.HybridMemorySegment;
+import org.apache.flink.core.memory.MemorySegment;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Random;
+
+public class LongSerializationSpeedBenchmark {
+
+ public static void main(String[] args) throws Exception {
+
+ final int LARGE_SEGMENT_SIZE = 1024 * 1024 * 1024;
+ final byte[] largeSegment = new byte[LARGE_SEGMENT_SIZE];
+ final ByteBuffer largeOffHeap = ByteBuffer.allocateDirect(LARGE_SEGMENT_SIZE);
+
+ fillOnHeap(largeSegment, (byte) -1);
+ fillOffHeap(largeOffHeap, (byte) -1);
+
+ final MemorySegment coreHeap = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(largeSegment, null);
+ final MemorySegment coreHybridOnHeap = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(largeSegment, null);
+ final MemorySegment coreHybridOffHeap = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(largeOffHeap, null);
+ final PureHeapMemorySegment pureHeap = new PureHeapMemorySegment(largeSegment);
+ final PureHybridMemorySegment pureHybridOnHeap = new PureHybridMemorySegment(largeSegment);
+ final PureHybridMemorySegment pureHybridOffHeap = new PureHybridMemorySegment(largeOffHeap);
+
+ final LongSerializer ser = LongSerializer.INSTANCE;
+
+ final long innerRounds = LARGE_SEGMENT_SIZE / 8;
+ final int outerRounds = 10;
+
+ {
+ System.out.println("testing core heap memory segment");
+
+ ArrayList<MemorySegment> memory = new ArrayList<>();
+ memory.add(coreHeap);
+ ArrayList<MemorySegment> target = new ArrayList<>();
+
+ CoreMemorySegmentOutView output = new CoreMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+ long start = System.nanoTime();
+ for (int outer = 0; outer < outerRounds; outer++) {
+ for (long i = 0; i < innerRounds; i++) {
+ ser.serialize(i, output);
+ }
+
+ target.clear();
+ memory.add(coreHeap);
+ output.reset();
+ }
+ long stop = System.nanoTime();
+
+ System.out.println(String.format("Core heap memory segment took %,d msecs", (stop - start) / 1000000));
+ }
+
+ {
+ System.out.println("testing core hybrid on heap memory segment");
+
+ ArrayList<MemorySegment> memory = new ArrayList<>();
+ memory.add(coreHybridOnHeap);
+ ArrayList<MemorySegment> target = new ArrayList<>();
+
+ CoreMemorySegmentOutView output = new CoreMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+ long start = System.nanoTime();
+ for (int outer = 0; outer < outerRounds; outer++) {
+ for (long i = 0; i < innerRounds; i++) {
+ ser.serialize(i, output);
+ }
+
+ target.clear();
+ memory.add(coreHybridOnHeap);
+ output.reset();
+ }
+ long stop = System.nanoTime();
+
+ System.out.println(String.format("Core hybrid on heap memory segment took %,d msecs", (stop - start) / 1000000));
+ }
+
+ {
+ System.out.println("testing core hybrid off heap memory segment");
+
+ ArrayList<MemorySegment> memory = new ArrayList<>();
+ memory.add(coreHybridOffHeap);
+ ArrayList<MemorySegment> target = new ArrayList<>();
+
+ CoreMemorySegmentOutView output = new CoreMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+ long start = System.nanoTime();
+ for (int outer = 0; outer < outerRounds; outer++) {
+ for (long i = 0; i < innerRounds; i++) {
+ ser.serialize(i, output);
+ }
+
+ target.clear();
+ memory.add(coreHybridOffHeap);
+ output.reset();
+ }
+ long stop = System.nanoTime();
+
+ System.out.println(String.format("Core hybrid off heap memory segment took %,d msecs", (stop - start) / 1000000));
+ }
+
+ {
+ System.out.println("testing pure heap memory segment");
+
+ ArrayList<PureHeapMemorySegment> memory = new ArrayList<>();
+ memory.add(pureHeap);
+ ArrayList<PureHeapMemorySegment> target = new ArrayList<>();
+
+ PureHeapMemorySegmentOutView output = new PureHeapMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+ long start = System.nanoTime();
+ for (int outer = 0; outer < outerRounds; outer++) {
+ for (long i = 0; i < innerRounds; i++) {
+ ser.serialize(i, output);
+ }
+
+ target.clear();
+ memory.add(pureHeap);
+ output.reset();
+ }
+ long stop = System.nanoTime();
+
+ System.out.println(String.format("Pure heap memory segment took %,d msecs", (stop - start) / 1000000));
+ }
+
+ {
+ System.out.println("testing pure hybrid memory segment on heap");
+
+ ArrayList<PureHybridMemorySegment> memory = new ArrayList<>();
+ memory.add(pureHybridOnHeap);
+ ArrayList<PureHybridMemorySegment> target = new ArrayList<>();
+
+ PureHybridMemorySegmentOutView output = new PureHybridMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+ long start = System.nanoTime();
+ for (int outer = 0; outer < outerRounds; outer++) {
+ for (long i = 0; i < innerRounds; i++) {
+ ser.serialize(i, output);
+ }
+
+ target.clear();
+ memory.add(pureHybridOnHeap);
+ output.reset();
+ }
+ long stop = System.nanoTime();
+
+ System.out.println(String.format("Pure hybrid on heap memory segment took %,d msecs", (stop - start) / 1000000));
+ }
+
+ {
+ System.out.println("testing pure hybrid memory segment off heap");
+
+ ArrayList<PureHybridMemorySegment> memory = new ArrayList<>();
+ memory.add(pureHybridOffHeap);
+ ArrayList<PureHybridMemorySegment> target = new ArrayList<>();
+
+ PureHybridMemorySegmentOutView output = new PureHybridMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+ long start = System.nanoTime();
+ for (int outer = 0; outer < outerRounds; outer++) {
+ for (long i = 0; i < innerRounds; i++) {
+ ser.serialize(i, output);
+ }
+
+ target.clear();
+ memory.add(pureHybridOffHeap);
+ output.reset();
+ }
+ long stop = System.nanoTime();
+
+ System.out.println(String.format("Pure hybrid off heap memory segment took %,d msecs", (stop - start) / 1000000));
+ }
+ }
+
+ private static String[] generateRandomStrings(long seed, int num, int maxLen, int minLen, boolean asciiOnly) {
+ Random rnd = new Random(seed);
+ String[] array = new String[num];
+ StringBuilder bld = new StringBuilder(maxLen);
+
+ int minCharValue = 40;
+ int charRange = asciiOnly ? 60 : 30000;
+
+ for (int i = 0; i < num; i++) {
+ bld.setLength(0);
+ int len = rnd.nextInt(maxLen - minLen) + minLen;
+
+ for (int k = 0; k < len; k++) {
+ bld.append((char) (rnd.nextInt(charRange) + minCharValue));
+ }
+
+ array[i] = bld.toString();
+ }
+
+ return array;
+ }
+
+ private static void fillOnHeap(byte[] buffer, byte data) {
+ for (int i = 0; i < buffer.length; i++) {
+ buffer[i] = data;
+ }
+ }
+
+ private static void fillOffHeap(ByteBuffer buffer, byte data) {
+ final int len = buffer.capacity();
+ for (int i = 0; i < len; i++) {
+ buffer.put(i, data);
+ }
+ }
+}
[07/15] flink git commit: [FLINK-1320] [core] Add an off-heap variant
of the managed memory
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/UnsafeMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/UnsafeMemorySegment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/UnsafeMemorySegment.java
deleted file mode 100644
index 21d3e27..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/UnsafeMemorySegment.java
+++ /dev/null
@@ -1,391 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.runtime.memorymanager;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-
-import org.apache.flink.core.memory.MemoryUtils;
-
-/**
- * This class represents a piece of memory allocated from the memory manager. The segment is backed
- * by a byte array and features random put and get methods for the basic types that are stored in a byte-wise
- * fashion in the memory.
- */
-public class UnsafeMemorySegment {
-
- // flag to enable / disable boundary checks. Note that the compiler eliminates the check code
- // paths (as dead code) when this constant is set to false.
- private static final boolean CHECKED = false;
-
- /**
- * The array in which the data is stored.
- */
- protected byte[] memory;
-
- /**
- * Wrapper for I/O requests.
- */
- protected ByteBuffer wrapper;
-
- // -------------------------------------------------------------------------
- // Constructors
- // -------------------------------------------------------------------------
-
- /**
- * Creates a new memory segment of given size with the provided views.
- */
- public UnsafeMemorySegment(byte[] memory) {
- this.memory = memory;
- }
-
- // -------------------------------------------------------------------------
- // MemorySegment Accessors
- // -------------------------------------------------------------------------
-
- /**
- * Checks whether this memory segment has already been freed. In that case, the
- * segment must not be used any more.
- *
- * @return True, if the segment has been freed, false otherwise.
- */
- public final boolean isFreed() {
- return this.memory == null;
- }
-
- /**
- * Gets the size of the memory segment, in bytes. Because segments
- * are backed by arrays, they cannot be larger than two GiBytes.
- *
- * @return The size in bytes.
- */
- public final int size() {
- return this.memory.length;
- }
-
- /**
- * Gets the byte array that backs the memory segment and this random access view.
- * Since different regions of the backing array are used by different segments, the logical
- * positions in this view do not correspond to the indexes in the backing array and need
- * to be translated via the {@link #translateOffset(int)} method.
- *
- * @return The backing byte array.
- */
- @Deprecated
- public final byte[] getBackingArray() {
- return this.memory;
- }
-
- /**
- * Translates the given offset for this view into the offset for the backing array.
- *
- * @param offset The offset to be translated.
- * @return The corresponding position in the backing array.
- */
- @Deprecated
- public final int translateOffset(int offset) {
- return offset;
- }
-
- // -------------------------------------------------------------------------
- // Helper methods
- // -------------------------------------------------------------------------
-
-
- /**
- * Wraps the chunk of the underlying memory located between <tt>offset<tt> and
- * <tt>length</tt> in a NIO ByteBuffer.
- *
- * @param offset The offset in the memory segment.
- * @param length The number of bytes to be wrapped as a buffer.
- * @return A <tt>ByteBuffer</tt> backed by the specified portion of the memory segment.
- * @throws IndexOutOfBoundsException Thrown, if offset is negative or larger than the memory segment size,
- * or if the offset plus the length is larger than the segment size.
- */
- public ByteBuffer wrap(int offset, int length) {
- if (offset > this.memory.length || offset > this.memory.length - length) {
- throw new IndexOutOfBoundsException();
- }
-
- if (this.wrapper == null) {
- this.wrapper = ByteBuffer.wrap(this.memory, offset, length);
- }
- else {
- this.wrapper.position(offset);
- this.wrapper.limit(offset + length);
- }
-
- return this.wrapper;
- }
-
-
- // --------------------------------------------------------------------
- // Random Access
- // --------------------------------------------------------------------
-
- // ------------------------------------------------------------------------------------------------------
- // WARNING: Any code for range checking must take care to avoid integer overflows. The position
- // integer may go up to <code>Integer.MAX_VALUE</tt>. Range checks that work after the principle
- // <code>position + 3 < end</code> may fail because <code>position + 3</code> becomes negative.
- // A safe solution is to subtract the delta from the limit, for example
- // <code>position < end - 3</code>. Since all indices are always positive, and the integer domain
- // has one more negative value than positive values, this can never cause an underflow.
- // ------------------------------------------------------------------------------------------------------
-
- public final byte get(int index) {
- return this.memory[index];
- }
-
- public final void put(int index, byte b) {
- this.memory[index] = b;
- }
-
- public final void get(int index, byte[] dst) {
- get(index, dst, 0, dst.length);
- }
-
- public final void put(int index, byte[] src) {
- put(index, src, 0, src.length);
- }
-
- public final void get(int index, byte[] dst, int offset, int length) {
- System.arraycopy(this.memory, index, dst, offset, length);
- }
-
- public final void put(int index, byte[] src, int offset, int length) {
- System.arraycopy(src, offset, this.memory, index, length);
- }
-
- public final void get(DataOutput out, int offset, int length) throws IOException {
- out.write(this.memory, offset, length);
- }
-
- public final void put(DataInput in, int offset, int length) throws IOException {
- in.readFully(this.memory, offset, length);
- }
-
- public final boolean getBoolean(int index) {
- return this.memory[index] != 0;
- }
-
- public final void putBoolean(int index, boolean value) {
- this.memory[index] = (byte) (value ? 1 : 0);
- }
-
- public final char getChar(int index) {
- return (char) ( ((this.memory[index ] & 0xff) << 8) |
- (this.memory[index + 1] & 0xff) );
- }
-
- public final void putChar(int index, char value) {
- this.memory[index ] = (byte) (value >> 8);
- this.memory[index + 1] = (byte) value;
- }
-
- public final short getShort(int index) {
- return (short) (
- ((this.memory[index ] & 0xff) << 8) |
- ((this.memory[index + 1] & 0xff)) );
- }
-
- public final void putShort(int index, short value) {
- this.memory[index ] = (byte) (value >> 8);
- this.memory[index + 1] = (byte) value;
- }
-
- @SuppressWarnings("restriction")
- public final int getInt(int index) {
- if (CHECKED) {
- if (index >= 0 && index <= this.memory.length - 4) {
- return UNSAFE.getInt(this.memory, BASE_OFFSET + index);
- } else {
- throw new IndexOutOfBoundsException();
- }
- } else {
- return UNSAFE.getInt(this.memory, BASE_OFFSET + index);
- }
- }
-
- public final int getIntLittleEndian(int index) {
- if (LITTLE_ENDIAN) {
- return getInt(index);
- } else {
- return Integer.reverseBytes(getInt(index));
- }
- }
-
- public final int getIntBigEndian(int index) {
- if (LITTLE_ENDIAN) {
- return Integer.reverseBytes(getInt(index));
- } else {
- return getInt(index);
- }
- }
-
- @SuppressWarnings("restriction")
- public final void putInt(int index, int value) {
- if (CHECKED) {
- if (index >= 0 && index <= this.memory.length - 4) {
- UNSAFE.putInt(this.memory, BASE_OFFSET + index, value);
- } else {
- throw new IndexOutOfBoundsException();
- }
- } else {
- UNSAFE.putInt(this.memory, BASE_OFFSET + index, value);
- }
- }
-
- public final void putIntLittleEndian(int index, int value) {
- if (LITTLE_ENDIAN) {
- putInt(index, value);
- } else {
- putInt(index, Integer.reverseBytes(value));
- }
- }
-
- public final void putIntBigEndian(int index, int value) {
- if (LITTLE_ENDIAN) {
- putInt(index, Integer.reverseBytes(value));
- } else {
- putInt(index, value);
- }
- }
-
- @SuppressWarnings("restriction")
- public final long getLong(int index) {
- if (CHECKED) {
- if (index >= 0 && index <= this.memory.length - 8) {
- return UNSAFE.getLong(this.memory, BASE_OFFSET + index);
- } else {
- throw new IndexOutOfBoundsException();
- }
- } else {
- return UNSAFE.getLong(this.memory, BASE_OFFSET + index);
- }
- }
-
- public final long getLongLittleEndian(int index) {
- if (LITTLE_ENDIAN) {
- return getLong(index);
- } else {
- return Long.reverseBytes(getLong(index));
- }
- }
-
- public final long getLongBigEndian(int index) {
- if (LITTLE_ENDIAN) {
- return Long.reverseBytes(getLong(index));
- } else {
- return getLong(index);
- }
- }
-
- @SuppressWarnings("restriction")
- public final void putLong(int index, long value) {
- if (CHECKED) {
- if (index >= 0 && index <= this.memory.length - 8) {
- UNSAFE.putLong(this.memory, BASE_OFFSET + index, value);
- } else {
- throw new IndexOutOfBoundsException();
- }
- } else {
- UNSAFE.putLong(this.memory, BASE_OFFSET + index, value);
- }
- }
-
- public final void putLongLittleEndian(int index, long value) {
- if (LITTLE_ENDIAN) {
- putLong(index, value);
- } else {
- putLong(index, Long.reverseBytes(value));
- }
- }
-
- public final void putLongBigEndian(int index, long value) {
- if (LITTLE_ENDIAN) {
- putLong(index, Long.reverseBytes(value));
- } else {
- putLong(index, value);
- }
- }
-
- public final float getFloat(int index) {
- return Float.intBitsToFloat(getInt(index));
- }
-
- public final float getFloatLittleEndian(int index) {
- return Float.intBitsToFloat(getIntLittleEndian(index));
- }
-
- public final float getFloatBigEndian(int index) {
- return Float.intBitsToFloat(getIntBigEndian(index));
- }
-
- public final void putFloat(int index, float value) {
- putInt(index, Float.floatToRawIntBits(value));
- }
-
- public final void putFloatLittleEndian(int index, float value) {
- putIntLittleEndian(index, Float.floatToRawIntBits(value));
- }
-
- public final void putFloatBigEndian(int index, float value) {
- putIntBigEndian(index, Float.floatToRawIntBits(value));
- }
-
- public final double getDouble(int index) {
- return Double.longBitsToDouble(getLong(index));
- }
-
- public final double getDoubleLittleEndian(int index) {
- return Double.longBitsToDouble(getLongLittleEndian(index));
- }
-
- public final double getDoubleBigEndian(int index) {
- return Double.longBitsToDouble(getLongBigEndian(index));
- }
-
- public final void putDouble(int index, double value) {
- putLong(index, Double.doubleToRawLongBits(value));
- }
-
- public final void putDoubleLittleEndian(int index, double value) {
- putLongLittleEndian(index, Double.doubleToRawLongBits(value));
- }
-
- public final void putDoubleBigEndian(int index, double value) {
- putLongBigEndian(index, Double.doubleToRawLongBits(value));
- }
-
- // --------------------------------------------------------------------------------------------
- // Utilities for native memory accesses and checks
- // --------------------------------------------------------------------------------------------
-
- @SuppressWarnings("restriction")
- private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
-
- @SuppressWarnings("restriction")
- private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
-
- private static final boolean LITTLE_ENDIAN = (MemoryUtils.NATIVE_BYTE_ORDER == ByteOrder.LITTLE_ENDIAN);
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
index 89b7709..38c74e0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.util.JoinTaskIterator;
import org.apache.flink.runtime.operators.util.TaskConfig;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
index cc39341..011c980 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
@@ -24,7 +24,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.flink.api.common.functions.CrossFunction;
import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.resettable.BlockResettableMutableObjectIterator;
import org.apache.flink.runtime.operators.resettable.SpillingResettableMutableObjectIterator;
import org.apache.flink.runtime.operators.util.TaskConfig;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java
index 998064a..30786aa 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
import org.apache.flink.runtime.operators.sort.NonReusingMergeOuterJoinIterator;
import org.apache.flink.runtime.operators.sort.ReusingMergeOuterJoinIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java
index 2bf778e..028ed95 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.sort.FixedLengthRecordSorter;
import org.apache.flink.runtime.operators.sort.InMemorySorter;
import org.apache.flink.runtime.operators.sort.NormalizedKeySorter;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
index 95e98ce..811f00c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
@@ -25,7 +25,7 @@ import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.hash.NonReusingBuildFirstHashMatchIterator;
import org.apache.flink.runtime.operators.hash.NonReusingBuildSecondHashMatchIterator;
import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashMatchIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/LeftOuterJoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/LeftOuterJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/LeftOuterJoinDriver.java
index b8932af..3cccab8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/LeftOuterJoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/LeftOuterJoinDriver.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
import org.apache.flink.runtime.operators.sort.NonReusingMergeOuterJoinIterator;
import org.apache.flink.runtime.operators.sort.ReusingMergeOuterJoinIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java
index 5c2ed67..baeda3a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.util.TaskConfig;
import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java
index f3ea700..19557bc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java
@@ -30,7 +30,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.sort.FixedLengthRecordSorter;
import org.apache.flink.runtime.operators.sort.InMemorySorter;
import org.apache.flink.runtime.operators.sort.NormalizedKeySorter;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java
index 3cefbba..6d35f92 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java
@@ -42,7 +42,7 @@ import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.chaining.ChainedDriver;
import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException;
import org.apache.flink.runtime.operators.resettable.SpillingResettableMutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RightOuterJoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RightOuterJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RightOuterJoinDriver.java
index c455dc4..c93637e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RightOuterJoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RightOuterJoinDriver.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
import org.apache.flink.runtime.operators.sort.NonReusingMergeOuterJoinIterator;
import org.apache.flink.runtime.operators.sort.ReusingMergeOuterJoinIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TempBarrier.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TempBarrier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TempBarrier.java
index 63f2fd5..59a0f59 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TempBarrier.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TempBarrier.java
@@ -30,9 +30,9 @@ import org.apache.flink.runtime.io.disk.InputViewIterator;
import org.apache.flink.runtime.io.disk.SpillingBuffer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.util.CloseableInputProvider;
import org.apache.flink.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/GroupCombineChainedDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/GroupCombineChainedDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/GroupCombineChainedDriver.java
index ff397c2..09f4288 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/GroupCombineChainedDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/GroupCombineChainedDriver.java
@@ -28,7 +28,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.RegularPactTask;
import org.apache.flink.runtime.operators.sort.FixedLengthRecordSorter;
import org.apache.flink.runtime.operators.sort.InMemorySorter;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java
index 4116145..75c1eed 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java
@@ -31,7 +31,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.RegularPactTask;
import org.apache.flink.runtime.operators.sort.FixedLengthRecordSorter;
import org.apache.flink.runtime.operators.sort.InMemorySorter;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java
index ff6548e..5a0c6cc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java
@@ -30,7 +30,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypePairComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
import org.apache.flink.runtime.util.IntArrayList;
import org.apache.flink.runtime.util.LongArrayList;
import org.apache.flink.runtime.util.MathUtils;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java
index 08b6191..3b12c68 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java
@@ -23,8 +23,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import java.util.List;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java
index b899acd..7baaee7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java
@@ -35,8 +35,8 @@ import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter;
import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
import org.apache.flink.runtime.util.MathUtils;
import org.apache.flink.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java
index ffb66fc..7554bc3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java
@@ -28,9 +28,9 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.core.memory.MemorySegmentSource;
import org.apache.flink.core.memory.SeekableDataInputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
/**
* In-memory partition with overflow buckets for {@link CompactingHashTable}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java
index 5000dab..dbdb5b2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java
@@ -26,8 +26,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.util.JoinTaskIterator;
import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java
index af1626a..b51c3b1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java
@@ -25,8 +25,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.util.MutableObjectIterator;
import java.io.IOException;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java
index 83952c9..26dba7b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java
@@ -26,8 +26,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.util.JoinTaskIterator;
import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java
index 029be5f..92b0fff 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java
@@ -25,8 +25,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.util.MutableObjectIterator;
import java.io.IOException;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java
index b4aaa95..65dfd89 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java
@@ -29,8 +29,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.util.JoinTaskIterator;
import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java
index 714a1f5..5635865 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java
@@ -28,8 +28,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.util.MutableObjectIterator;
public class ReusingBuildFirstReOpenableHashMatchIterator<V1, V2, O> extends ReusingBuildFirstHashMatchIterator<V1, V2, O> {
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java
index b7c3e29..156f259 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java
@@ -29,8 +29,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.util.JoinTaskIterator;
import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java
index 4b4cdf5..a0791fe 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java
@@ -28,8 +28,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.util.MutableObjectIterator;
public class ReusingBuildSecondReOpenableHashMatchIterator<V1, V2, O> extends ReusingBuildSecondHashMatchIterator<V1, V2, O> {
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/AbstractBlockResettableIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/AbstractBlockResettableIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/AbstractBlockResettableIterator.java
index 730d19a..859757f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/AbstractBlockResettableIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/AbstractBlockResettableIterator.java
@@ -30,9 +30,9 @@ import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.RandomAccessInputView;
import org.apache.flink.runtime.io.disk.SimpleCollectingOutputView;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.util.MemoryBlockIterator;
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIterator.java
index abce462..124ba55 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIterator.java
@@ -25,8 +25,8 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.util.ResettableMutableObjectIterator;
import org.apache.flink.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIterator.java
index 9d581ce..408d6e4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIterator.java
@@ -27,8 +27,8 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.util.ResettableIterator;
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIterator.java
index baa0fb2..ff58092 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIterator.java
@@ -21,8 +21,8 @@ package org.apache.flink.runtime.operators.resettable;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIterator.java
index e5a4501..c59b52a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIterator.java
@@ -33,9 +33,9 @@ import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.SpillingBuffer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.util.ResettableIterator;
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIterator.java
index 5467ae9..12d5400 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIterator.java
@@ -31,9 +31,9 @@ import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.SpillingBuffer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.util.ResettableMutableObjectIterator;
import org.apache.flink.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java
index e9ccf52..db56227 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java
@@ -24,8 +24,8 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java
index 68e0d47..a6bd4d0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java
@@ -25,8 +25,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.operators.resettable.NonReusingBlockResettableIterator;
import org.apache.flink.runtime.operators.resettable.SpillingResettableIterator;
import org.apache.flink.runtime.operators.util.JoinTaskIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java
index f34639f..d109cf8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java
@@ -24,8 +24,8 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.util.Collector;
import org.apache.flink.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java
index 8da9413..f662a7e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java
@@ -30,8 +30,8 @@ import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView;
import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.util.EmptyMutableObjectIterator;
import org.apache.flink.runtime.util.ReusingKeyGroupedIterator;
import org.apache.flink.util.Collector;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java
index cd982c4..e932592 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java
@@ -28,8 +28,8 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
import org.apache.flink.util.MutableObjectIterator;
/**
@@ -259,11 +259,11 @@ public final class FixedLengthRecordSorter<T> implements InMemorySorter<T> {
// Access Utilities
// ------------------------------------------------------------------------
- private final boolean memoryAvailable() {
+ private boolean memoryAvailable() {
return !this.freeMemory.isEmpty();
}
- private final MemorySegment nextMemorySegment() {
+ private MemorySegment nextMemorySegment() {
return this.freeMemory.remove(this.freeMemory.size() - 1);
}
@@ -282,7 +282,7 @@ public final class FixedLengthRecordSorter<T> implements InMemorySorter<T> {
final MemorySegment segI = this.sortBuffer.get(bufferNumI);
final MemorySegment segJ = this.sortBuffer.get(bufferNumJ);
- int val = MemorySegment.compare(segI, segJ, segmentOffsetI, segmentOffsetJ, this.numKeyBytes);
+ int val = segI.compare(segJ, segmentOffsetI, segmentOffsetJ, this.numKeyBytes);
return this.useNormKeyUninverted ? val : -val;
}
@@ -297,7 +297,7 @@ public final class FixedLengthRecordSorter<T> implements InMemorySorter<T> {
final MemorySegment segI = this.sortBuffer.get(bufferNumI);
final MemorySegment segJ = this.sortBuffer.get(bufferNumJ);
- MemorySegment.swapBytes(segI, segJ, this.swapBuffer, segmentOffsetI, segmentOffsetJ, this.recordSize);
+ segI.swapBytes(this.swapBuffer, segJ, segmentOffsetI, segmentOffsetJ, this.recordSize);
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/LargeRecordHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/LargeRecordHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/LargeRecordHandler.java
index ce63021..e4a99fb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/LargeRecordHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/LargeRecordHandler.java
@@ -44,7 +44,7 @@ import org.apache.flink.runtime.io.disk.SeekableFileChannelInputView;
import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.types.NullKeyFieldException;
import org.apache.flink.util.MutableObjectIterator;
import org.slf4j.Logger;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeInnerJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeInnerJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeInnerJoinIterator.java
index 644084c..d208398 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeInnerJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeInnerJoinIterator.java
@@ -23,8 +23,8 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.util.KeyGroupedIterator;
import org.apache.flink.runtime.util.NonReusingKeyGroupedIterator;
import org.apache.flink.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java
index 33510d3..db47f16 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java
@@ -23,8 +23,8 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.util.KeyGroupedIterator;
import org.apache.flink.runtime.util.NonReusingKeyGroupedIterator;
import org.apache.flink.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java
index fe87788..c05e518 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java
@@ -29,7 +29,7 @@ import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.disk.RandomAccessInputView;
import org.apache.flink.runtime.io.disk.SimpleCollectingOutputView;
import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
import org.apache.flink.util.MutableObjectIterator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -322,7 +322,7 @@ public final class NormalizedKeySorter<T> implements InMemorySorter<T> {
// Access Utilities
// ------------------------------------------------------------------------
- private final long readPointer(int logicalPosition) {
+ private long readPointer(int logicalPosition) {
if (logicalPosition < 0 | logicalPosition >= this.numRecords) {
throw new IndexOutOfBoundsException();
}
@@ -333,17 +333,17 @@ public final class NormalizedKeySorter<T> implements InMemorySorter<T> {
return (this.sortIndex.get(bufferNum).getLong(segmentOffset * this.indexEntrySize)) & POINTER_MASK;
}
- private final T getRecordFromBuffer(T reuse, long pointer) throws IOException {
+ private T getRecordFromBuffer(T reuse, long pointer) throws IOException {
this.recordBuffer.setReadPosition(pointer);
return this.serializer.deserialize(reuse, this.recordBuffer);
}
- private final T getRecordFromBuffer(long pointer) throws IOException {
+ private T getRecordFromBuffer(long pointer) throws IOException {
this.recordBuffer.setReadPosition(pointer);
return this.serializer.deserialize(this.recordBuffer);
}
- private final int compareRecords(long pointer1, long pointer2) {
+ private int compareRecords(long pointer1, long pointer2) {
this.recordBuffer.setReadPosition(pointer1);
this.recordBufferForComparison.setReadPosition(pointer2);
@@ -354,11 +354,11 @@ public final class NormalizedKeySorter<T> implements InMemorySorter<T> {
}
}
- private final boolean memoryAvailable() {
+ private boolean memoryAvailable() {
return !this.freeMemory.isEmpty();
}
- private final MemorySegment nextMemorySegment() {
+ private MemorySegment nextMemorySegment() {
return this.freeMemory.remove(this.freeMemory.size() - 1);
}
@@ -377,7 +377,7 @@ public final class NormalizedKeySorter<T> implements InMemorySorter<T> {
final MemorySegment segI = this.sortIndex.get(bufferNumI);
final MemorySegment segJ = this.sortIndex.get(bufferNumJ);
- int val = MemorySegment.compare(segI, segJ, segmentOffsetI + OFFSET_LEN, segmentOffsetJ + OFFSET_LEN, this.numKeyBytes);
+ int val = segI.compare(segJ, segmentOffsetI + OFFSET_LEN, segmentOffsetJ + OFFSET_LEN, this.numKeyBytes);
if (val != 0 || this.normalizedKeyFullyDetermines) {
return this.useNormKeyUninverted ? val : -val;
@@ -400,7 +400,7 @@ public final class NormalizedKeySorter<T> implements InMemorySorter<T> {
final MemorySegment segI = this.sortIndex.get(bufferNumI);
final MemorySegment segJ = this.sortIndex.get(bufferNumJ);
- MemorySegment.swapBytes(segI, segJ, this.swapBuffer, segmentOffsetI, segmentOffsetJ, this.indexEntrySize);
+ segI.swapBytes(this.swapBuffer, segJ, segmentOffsetI, segmentOffsetJ, this.indexEntrySize);
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeInnerJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeInnerJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeInnerJoinIterator.java
index 3a1a17a..6b8fcf9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeInnerJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeInnerJoinIterator.java
@@ -23,8 +23,8 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.util.KeyGroupedIterator;
import org.apache.flink.runtime.util.ReusingKeyGroupedIterator;
import org.apache.flink.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java
index ce7bab4..8382b86 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java
@@ -23,8 +23,8 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.util.KeyGroupedIterator;
import org.apache.flink.runtime.util.ReusingKeyGroupedIterator;
import org.apache.flink.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
index 388b7b9..fd1062d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
@@ -46,8 +46,8 @@ import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel.ID;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.util.EmptyMutableObjectIterator;
import org.apache.flink.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/CoGroupTaskIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/CoGroupTaskIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/CoGroupTaskIterator.java
index 6152797..ac3668b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/CoGroupTaskIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/CoGroupTaskIterator.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.operators.util;
import java.io.IOException;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
/**
* Interface describing the methods that have to be implemented by local strategies for the CoGroup Pact.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java
index 1f1d625..197eaf8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java
@@ -23,7 +23,7 @@ package org.apache.flink.runtime.operators.util;
import java.io.IOException;
import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
import org.apache.flink.util.Collector;
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java
index 07cea33..567f051 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java
@@ -31,7 +31,7 @@ import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
import org.apache.flink.runtime.state.StateHandle;
import org.apache.flink.util.SerializedValue;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index a19a57a..0d1dd13 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -49,7 +49,7 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
import org.apache.flink.runtime.jobgraph.tasks.CheckpointNotificationOperator;
import org.apache.flink.runtime.jobgraph.tasks.CheckpointedOperator;
import org.apache.flink.runtime.jobgraph.tasks.OperatorStateCarrier;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.messages.TaskManagerMessages.FatalError;
import org.apache.flink.runtime.messages.TaskMessages.TaskInFinalState;
import org.apache.flink.runtime.messages.TaskMessages.UpdateTaskExecutionState;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java
index f243027..b8519b0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java
@@ -116,7 +116,24 @@ public class EnvironmentInformation {
* @return The maximum JVM heap size, in bytes.
*/
public static long getMaxJvmHeapMemory() {
- return Runtime.getRuntime().maxMemory();
+ long maxMemory = Runtime.getRuntime().maxMemory();
+
+ if (maxMemory == Long.MAX_VALUE) {
+ // amount of free memory unknown
+ try {
+ // workaround for Oracle JDK
+ OperatingSystemMXBean operatingSystemMXBean = ManagementFactory.getOperatingSystemMXBean();
+ Class<?> clazz = Class.forName("com.sun.management.OperatingSystemMXBean");
+ Method method = clazz.getMethod("getTotalPhysicalMemorySize");
+ maxMemory = (Long) method.invoke(operatingSystemMXBean) / 4;
+ }
+ catch (Throwable e) {
+ throw new RuntimeException("Could not determine the amount of free memory.\n" +
+ "Please set the maximum memory for the JVM, e.g. -Xmx512M for 512 megabytes.");
+ }
+ }
+
+ return maxMemory;
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
index 5bcda60..065211c 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
@@ -18,12 +18,14 @@
package org.apache.flink.runtime.taskmanager
+import org.apache.flink.core.memory.MemoryType
import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode
import org.apache.flink.runtime.io.network.netty.NettyConfig
case class NetworkEnvironmentConfiguration(
numNetworkBuffers: Int,
networkBufferSize: Int,
+ memoryType: MemoryType,
ioMode: IOMode,
nettyConfig: Option[NettyConfig] = None,
- partitionRequestInitialAndMaxBackoff: Tuple2[Integer, Integer] = (500, 3000))
+ partitionRequestInitialAndMaxBackoff: (Integer, Integer) = (500, 3000))
[02/15] flink git commit: [FLINK-2636] [streaming] Create common type
StreamElement for StreamRecord and Watermark
Posted by se...@apache.org.
[FLINK-2636] [streaming] Create common type StreamElement for StreamRecord and Watermark
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c09d14a9
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c09d14a9
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c09d14a9
Branch: refs/heads/master
Commit: c09d14a9d836d099c32639180dd58216ae6149a4
Parents: 655a891
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Sep 8 15:21:23 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Sep 8 20:58:05 2015 +0200
----------------------------------------------------------------------
.../api/common/typeutils/TypeSerializer.java | 1 -
.../streaming/api/watermark/Watermark.java | 24 ++----
.../runtime/io/RecordWriterOutput.java | 13 ++--
.../runtime/io/StreamInputProcessor.java | 29 ++++---
.../runtime/io/StreamTwoInputProcessor.java | 63 ++++++++--------
.../MultiplexingStreamRecordSerializer.java | 79 +++++++++++---------
.../runtime/streamrecord/StreamElement.java | 62 +++++++++++++++
.../runtime/streamrecord/StreamRecord.java | 4 +-
.../consumer/StreamTestSingleInputGate.java | 5 +-
.../api/operators/StreamFilterTest.java | 1 -
.../api/operators/StreamGroupedFoldTest.java | 2 -
.../api/operators/StreamGroupedReduceTest.java | 1 -
.../streaming/api/operators/StreamMapTest.java | 1 -
.../api/operators/StreamProjectTest.java | 3 +-
.../api/operators/co/CoStreamFlatMapTest.java | 5 --
.../runtime/tasks/StreamTaskTestHarness.java | 6 +-
16 files changed, 173 insertions(+), 126 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
index 542b059..45b0669 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
@@ -16,7 +16,6 @@
* limitations under the License.
*/
-
package org.apache.flink.api.common.typeutils;
import java.io.IOException;
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java
index 1d88fe2..163791e 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java
@@ -17,6 +17,8 @@
*/
package org.apache.flink.streaming.api.watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+
/**
* A Watermark tells operators that receive it that no elements with a timestamp older or equal
* to the watermark timestamp should arrive at the operator. Watermarks are emitted at the
@@ -31,11 +33,11 @@ package org.apache.flink.streaming.api.watermark;
* In some cases a watermark is only a heuristic and operators should be able to deal with
* late elements. They can either discard those or update the result and emit updates/retractions
* to downstream operations.
- *
*/
-public class Watermark {
+public class Watermark extends StreamElement {
- private long timestamp;
+ /** The timestamp of the watermark */
+ private final long timestamp;
/**
* Creates a new watermark with the given timestamp.
@@ -53,16 +55,8 @@ public class Watermark {
@Override
public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- Watermark watermark = (Watermark) o;
-
- return timestamp == watermark.timestamp;
+ return this == o ||
+ o != null && o.getClass() == Watermark.class && ((Watermark) o).timestamp == this.timestamp;
}
@Override
@@ -72,8 +66,6 @@ public class Watermark {
@Override
public String toString() {
- return "Watermark{" +
- "timestamp=" + timestamp +
- '}';
+ return "Watermark @ " + timestamp;
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java
index 7048464..34e5800 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java
@@ -24,6 +24,7 @@ import org.apache.flink.runtime.event.AbstractEvent;
import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -36,9 +37,9 @@ import static com.google.common.base.Preconditions.checkNotNull;
*/
public class RecordWriterOutput<OUT> implements Output<StreamRecord<OUT>> {
- private StreamRecordWriter<SerializationDelegate<Object>> recordWriter;
+ private StreamRecordWriter<SerializationDelegate<StreamElement>> recordWriter;
- private SerializationDelegate<Object> serializationDelegate;
+ private SerializationDelegate<StreamElement> serializationDelegate;
@SuppressWarnings("unchecked")
@@ -51,19 +52,19 @@ public class RecordWriterOutput<OUT> implements Output<StreamRecord<OUT>> {
// generic hack: cast the writer to generic Object type so we can use it
// with multiplexed records and watermarks
- this.recordWriter = (StreamRecordWriter<SerializationDelegate<Object>>)
+ this.recordWriter = (StreamRecordWriter<SerializationDelegate<StreamElement>>)
(StreamRecordWriter<?>) recordWriter;
- TypeSerializer<Object> outRecordSerializer;
+ TypeSerializer<StreamElement> outRecordSerializer;
if (enableWatermarkMultiplexing) {
outRecordSerializer = new MultiplexingStreamRecordSerializer<OUT>(outSerializer);
} else {
- outRecordSerializer = (TypeSerializer<Object>)
+ outRecordSerializer = (TypeSerializer<StreamElement>)
(TypeSerializer<?>) new StreamRecordSerializer<OUT>(outSerializer);
}
if (outSerializer != null) {
- serializationDelegate = new SerializationDelegate<Object>(outRecordSerializer);
+ serializationDelegate = new SerializationDelegate<StreamElement>(outRecordSerializer);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
index de021ff..cc91d63 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
@@ -36,6 +36,7 @@ import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate;
import org.apache.flink.runtime.util.event.EventListener;
import org.apache.flink.streaming.api.CheckpointingMode;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -53,9 +54,9 @@ import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
*/
public class StreamInputProcessor<IN> extends AbstractReader implements StreamingReader {
- private final RecordDeserializer<DeserializationDelegate<Object>>[] recordDeserializers;
+ private final RecordDeserializer<DeserializationDelegate<StreamElement>>[] recordDeserializers;
- private RecordDeserializer<DeserializationDelegate<Object>> currentRecordDeserializer;
+ private RecordDeserializer<DeserializationDelegate<StreamElement>> currentRecordDeserializer;
// We need to keep track of the channel from which a buffer came, so that we can
// appropriately map the watermarks to input channels
@@ -68,9 +69,9 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
private final long[] watermarks;
private long lastEmittedWatermark;
- private final DeserializationDelegate<Object> deserializationDelegate;
+ private final DeserializationDelegate<StreamElement> deserializationDelegate;
- @SuppressWarnings("unchecked")
+ @SuppressWarnings({"unchecked", "rawtypes"})
public StreamInputProcessor(InputGate[] inputGates, TypeSerializer<IN> inputSerializer,
EventListener<CheckpointBarrier> checkpointListener,
CheckpointingMode checkpointMode,
@@ -95,10 +96,10 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
if (enableWatermarkMultiplexing) {
MultiplexingStreamRecordSerializer<IN> ser = new MultiplexingStreamRecordSerializer<IN>(inputSerializer);
- this.deserializationDelegate = new NonReusingDeserializationDelegate<Object>(ser);
+ this.deserializationDelegate = new NonReusingDeserializationDelegate<StreamElement>(ser);
} else {
StreamRecordSerializer<IN> ser = new StreamRecordSerializer<IN>(inputSerializer);
- this.deserializationDelegate = (NonReusingDeserializationDelegate<Object>)
+ this.deserializationDelegate = (NonReusingDeserializationDelegate<StreamElement>)
(NonReusingDeserializationDelegate<?>) new NonReusingDeserializationDelegate<StreamRecord<IN>>(ser);
}
@@ -106,7 +107,7 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate.getNumberOfInputChannels()];
for (int i = 0; i < recordDeserializers.length; i++) {
- recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer<DeserializationDelegate<Object>>();
+ recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer<DeserializationDelegate<StreamElement>>();
}
watermarks = new long[inputGate.getNumberOfInputChannels()];
@@ -132,18 +133,15 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
}
if (result.isFullRecord()) {
- Object recordOrWatermark = deserializationDelegate.getInstance();
+ StreamElement recordOrWatermark = deserializationDelegate.getInstance();
- if (recordOrWatermark instanceof Watermark) {
- Watermark mark = (Watermark) recordOrWatermark;
- long watermarkMillis = mark.getTimestamp();
+ if (recordOrWatermark.isWatermark()) {
+ long watermarkMillis = recordOrWatermark.asWatermark().getTimestamp();
if (watermarkMillis > watermarks[currentChannel]) {
watermarks[currentChannel] = watermarkMillis;
long newMinWatermark = Long.MAX_VALUE;
for (long watermark : watermarks) {
- if (watermark < newMinWatermark) {
- newMinWatermark = watermark;
- }
+ newMinWatermark = Math.min(watermark, newMinWatermark);
}
if (newMinWatermark > lastEmittedWatermark) {
lastEmittedWatermark = newMinWatermark;
@@ -154,8 +152,7 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
}
else {
// now we can do the actual processing
- @SuppressWarnings("unchecked")
- StreamRecord<IN> record = (StreamRecord<IN>) deserializationDelegate.getInstance();
+ StreamRecord<IN> record = recordOrWatermark.asRecord();
StreamingRuntimeContext ctx = streamOperator.getRuntimeContext();
if (ctx != null) {
ctx.setNextInput(record);
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
index e0af729..7dffa71 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
@@ -35,6 +35,7 @@ import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate;
import org.apache.flink.runtime.util.event.EventListener;
import org.apache.flink.streaming.api.CheckpointingMode;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -44,6 +45,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
+import java.util.Arrays;
import java.util.Collection;
/**
@@ -61,9 +63,9 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
@SuppressWarnings("unused")
private static final Logger LOG = LoggerFactory.getLogger(StreamTwoInputProcessor.class);
- private final RecordDeserializer<DeserializationDelegate<Object>>[] recordDeserializers;
+ private final RecordDeserializer<DeserializationDelegate<StreamElement>>[] recordDeserializers;
- private RecordDeserializer<DeserializationDelegate<Object>> currentRecordDeserializer;
+ private RecordDeserializer<DeserializationDelegate<StreamElement>> currentRecordDeserializer;
// We need to keep track of the channel from which a buffer came, so that we can
// appropriately map the watermarks to input channels
@@ -81,8 +83,8 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
private final int numInputChannels1;
- private final DeserializationDelegate<Object> deserializationDelegate1;
- private final DeserializationDelegate<Object> deserializationDelegate2;
+ private final DeserializationDelegate<StreamElement> deserializationDelegate1;
+ private final DeserializationDelegate<StreamElement> deserializationDelegate2;
@SuppressWarnings({"unchecked", "rawtypes"})
public StreamTwoInputProcessor(
@@ -113,21 +115,21 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
if (enableWatermarkMultiplexing) {
MultiplexingStreamRecordSerializer<IN1> ser = new MultiplexingStreamRecordSerializer<IN1>(inputSerializer1);
- this.deserializationDelegate1 = new NonReusingDeserializationDelegate<Object>(ser);
+ this.deserializationDelegate1 = new NonReusingDeserializationDelegate<StreamElement>(ser);
}
else {
StreamRecordSerializer<IN1> ser = new StreamRecordSerializer<IN1>(inputSerializer1);
- this.deserializationDelegate1 = (DeserializationDelegate<Object>)
+ this.deserializationDelegate1 = (DeserializationDelegate<StreamElement>)
(DeserializationDelegate<?>) new NonReusingDeserializationDelegate<StreamRecord<IN1>>(ser);
}
if (enableWatermarkMultiplexing) {
MultiplexingStreamRecordSerializer<IN2> ser = new MultiplexingStreamRecordSerializer<IN2>(inputSerializer2);
- this.deserializationDelegate2 = new NonReusingDeserializationDelegate<Object>(ser);
+ this.deserializationDelegate2 = new NonReusingDeserializationDelegate<StreamElement>(ser);
}
else {
StreamRecordSerializer<IN2> ser = new StreamRecordSerializer<IN2>(inputSerializer2);
- this.deserializationDelegate2 = (DeserializationDelegate<Object>)
+ this.deserializationDelegate2 = (DeserializationDelegate<StreamElement>)
(DeserializationDelegate<?>) new NonReusingDeserializationDelegate<StreamRecord<IN2>>(ser);
}
@@ -135,7 +137,7 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate.getNumberOfInputChannels()];
for (int i = 0; i < recordDeserializers.length; i++) {
- recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer<DeserializationDelegate<Object>>();
+ recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer<DeserializationDelegate<StreamElement>>();
}
// determine which unioned channels belong to input 1 and which belong to input 2
@@ -148,15 +150,11 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
int numInputChannels2 = inputGate.getNumberOfInputChannels() - numInputChannels1;
watermarks1 = new long[numInputChannels1];
- for (int i = 0; i < numInputChannels1; i++) {
- watermarks1[i] = Long.MIN_VALUE;
- }
+ Arrays.fill(watermarks1, Long.MIN_VALUE);
lastEmittedWatermark1 = Long.MIN_VALUE;
watermarks2 = new long[numInputChannels2];
- for (int i = 0; i < numInputChannels2; i++) {
- watermarks2[i] = Long.MIN_VALUE;
- }
+ Arrays.fill(watermarks2, Long.MIN_VALUE);
lastEmittedWatermark2 = Long.MIN_VALUE;
}
@@ -182,22 +180,25 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
if (result.isFullRecord()) {
if (currentChannel < numInputChannels1) {
- Object recordOrWatermark = deserializationDelegate1.getInstance();
- if (recordOrWatermark instanceof Watermark) {
+ StreamElement recordOrWatermark = deserializationDelegate1.getInstance();
+ if (recordOrWatermark.isWatermark()) {
handleWatermark(streamOperator, (Watermark) recordOrWatermark, currentChannel);
continue;
- } else {
- streamOperator.processElement1((StreamRecord<IN1>) deserializationDelegate1.getInstance());
+ }
+ else {
+ streamOperator.processElement1(recordOrWatermark.<IN1>asRecord());
return true;
}
- } else {
- Object recordOrWatermark = deserializationDelegate2.getInstance();
- if (recordOrWatermark instanceof Watermark) {
- handleWatermark(streamOperator, (Watermark) recordOrWatermark, currentChannel);
+ }
+ else {
+ StreamElement recordOrWatermark = deserializationDelegate2.getInstance();
+ if (recordOrWatermark.isWatermark()) {
+ handleWatermark(streamOperator, recordOrWatermark.asWatermark(), currentChannel);
continue;
- } else {
- streamOperator.processElement2((StreamRecord<IN2>) deserializationDelegate2.getInstance());
+ }
+ else {
+ streamOperator.processElement2(recordOrWatermark.<IN2>asRecord());
return true;
}
}
@@ -234,10 +235,8 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
if (watermarkMillis > watermarks1[channelIndex]) {
watermarks1[channelIndex] = watermarkMillis;
long newMinWatermark = Long.MAX_VALUE;
- for (long aWatermarks1 : watermarks1) {
- if (aWatermarks1 < newMinWatermark) {
- newMinWatermark = aWatermarks1;
- }
+ for (long wm : watermarks1) {
+ newMinWatermark = Math.min(wm, newMinWatermark);
}
if (newMinWatermark > lastEmittedWatermark1) {
lastEmittedWatermark1 = newMinWatermark;
@@ -250,10 +249,8 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
if (watermarkMillis > watermarks2[channelIndex]) {
watermarks2[channelIndex] = watermarkMillis;
long newMinWatermark = Long.MAX_VALUE;
- for (long aWatermarks2 : watermarks2) {
- if (aWatermarks2 < newMinWatermark) {
- newMinWatermark = aWatermarks2;
- }
+ for (long wm : watermarks2) {
+ newMinWatermark = Math.min(wm, newMinWatermark);
}
if (newMinWatermark > lastEmittedWatermark2) {
lastEmittedWatermark2 = newMinWatermark;
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
index 075c4fc..156e5d6 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
@@ -27,22 +27,22 @@ import org.apache.flink.streaming.api.watermark.Watermark;
import java.io.IOException;
/**
- * Serializer for {@link StreamRecord} and {@link Watermark}. This does not behave like a normal
+ * Serializer for {@link StreamRecord} and {@link org.apache.flink.streaming.api.watermark.Watermark}. This does not behave like a normal
* {@link TypeSerializer}, instead, this is only used at the
* {@link org.apache.flink.streaming.runtime.tasks.StreamTask} level for transmitting
- * {@link StreamRecord StreamRecords} and {@link Watermark Watermarks}. This serializer
+ * {@link StreamRecord StreamRecords} and {@link org.apache.flink.streaming.api.watermark.Watermark Watermarks}. This serializer
* can handle both of them, therefore it returns {@link Object} the result has
* to be cast to the correct type.
*
* @param <T> The type of value in the {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord}
*/
-public final class MultiplexingStreamRecordSerializer<T> extends TypeSerializer<Object> {
+public final class MultiplexingStreamRecordSerializer<T> extends TypeSerializer<StreamElement> {
private static final long serialVersionUID = 1L;
private static final long IS_WATERMARK = Long.MIN_VALUE;
- protected final TypeSerializer<T> typeSerializer;
+ private final TypeSerializer<T> typeSerializer;
public MultiplexingStreamRecordSerializer(TypeSerializer<T> serializer) {
@@ -59,87 +59,94 @@ public final class MultiplexingStreamRecordSerializer<T> extends TypeSerializer<
}
@Override
- public TypeSerializer<Object> duplicate() {
- return this;
+ public TypeSerializer<StreamElement> duplicate() {
+ TypeSerializer<T> copy = typeSerializer.duplicate();
+ return (copy == typeSerializer) ? this : new MultiplexingStreamRecordSerializer<T>(copy);
}
@Override
- public Object createInstance() {
+ public StreamRecord<T> createInstance() {
return new StreamRecord<T>(typeSerializer.createInstance(), 0L);
}
@Override
- @SuppressWarnings("unchecked")
- public Object copy(Object from) {
+ public StreamElement copy(StreamElement from) {
// we can reuse the timestamp since Instant is immutable
- if (from instanceof StreamRecord) {
- StreamRecord<T> fromRecord = (StreamRecord<T>) from;
+ if (from.isRecord()) {
+ StreamRecord<T> fromRecord = from.asRecord();
return new StreamRecord<T>(typeSerializer.copy(fromRecord.getValue()), fromRecord.getTimestamp());
- } else if (from instanceof Watermark) {
+ }
+ else if (from.isWatermark()) {
// is immutable
return from;
- } else {
+ }
+ else {
throw new RuntimeException("Cannot copy " + from);
}
}
@Override
- @SuppressWarnings("unchecked")
- public Object copy(Object from, Object reuse) {
- if (from instanceof StreamRecord && reuse instanceof StreamRecord) {
- StreamRecord<T> fromRecord = (StreamRecord<T>) from;
- StreamRecord<T> reuseRecord = (StreamRecord<T>) reuse;
+ public StreamElement copy(StreamElement from, StreamElement reuse) {
+ if (from.isRecord() && reuse.isRecord()) {
+ StreamRecord<T> fromRecord = from.asRecord();
+ StreamRecord<T> reuseRecord = reuse.asRecord();
- reuseRecord.replace(typeSerializer.copy(fromRecord.getValue(), reuseRecord.getValue()), fromRecord.getTimestamp());
+ T valueCopy = typeSerializer.copy(fromRecord.getValue(), reuseRecord.getValue());
+ reuseRecord.replace(valueCopy, fromRecord.getTimestamp());
return reuse;
- } else if (from instanceof Watermark) {
+ }
+ else if (from.isWatermark()) {
// is immutable
return from;
- } else {
- throw new RuntimeException("Cannot copy " + from);
+ }
+ else {
+ throw new RuntimeException("Cannot copy " + from + " -> " + reuse);
}
}
@Override
public int getLength() {
- return 0;
+ return -1;
}
@Override
- @SuppressWarnings("unchecked")
- public void serialize(Object value, DataOutputView target) throws IOException {
- if (value instanceof StreamRecord) {
- StreamRecord<T> record = (StreamRecord<T>) value;
+ public void serialize(StreamElement value, DataOutputView target) throws IOException {
+ if (value.isRecord()) {
+ StreamRecord<T> record = value.asRecord();
target.writeLong(record.getTimestamp());
typeSerializer.serialize(record.getValue(), target);
- } else if (value instanceof Watermark) {
+ }
+ else if (value.isWatermark()) {
target.writeLong(IS_WATERMARK);
- target.writeLong(((Watermark) value).getTimestamp());
+ target.writeLong(value.asWatermark().getTimestamp());
+ }
+ else {
+ throw new RuntimeException();
}
}
@Override
- public Object deserialize(DataInputView source) throws IOException {
+ public StreamElement deserialize(DataInputView source) throws IOException {
long millis = source.readLong();
if (millis == IS_WATERMARK) {
return new Watermark(source.readLong());
- } else {
+ }
+ else {
T element = typeSerializer.deserialize(source);
return new StreamRecord<T>(element, millis);
}
}
@Override
- @SuppressWarnings("unchecked")
- public Object deserialize(Object reuse, DataInputView source) throws IOException {
+ public StreamElement deserialize(StreamElement reuse, DataInputView source) throws IOException {
long millis = source.readLong();
if (millis == IS_WATERMARK) {
return new Watermark(source.readLong());
-
- } else {
- StreamRecord<T> reuseRecord = (StreamRecord<T>) reuse;
+ }
+ else {
+ StreamRecord<T> reuseRecord = reuse.asRecord();
T element = typeSerializer.deserialize(reuseRecord.getValue(), source);
reuseRecord.replace(element, millis);
return reuse;
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java
new file mode 100644
index 0000000..80df72e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.runtime.streamrecord;
+
+import org.apache.flink.streaming.api.watermark.Watermark;
+
+/**
+ * An element in a data stream. Can be a record or a Watermark.
+ */
+public abstract class StreamElement {
+
+ /**
+ * Checks whether this element is a watermark.
+ * @return True, if this element is a watermark, false otherwise.
+ */
+ public final boolean isWatermark() {
+ return getClass() == Watermark.class;
+ }
+
+ /**
+ * Checks whether this element is a record.
+ * @return True, if this element is a record, false otherwise.
+ */
+ public final boolean isRecord() {
+ return getClass() == StreamRecord.class;
+ }
+
+ /**
+ * Casts this element into a StreamRecord.
+ * @return This element as a stream record.
+ * @throws java.lang.ClassCastException Thrown, if this element is actually not a stream record.
+ */
+ @SuppressWarnings("unchecked")
+ public final <E> StreamRecord<E> asRecord() {
+ return (StreamRecord<E>) this;
+ }
+
+ /**
+ * Casts this element into a Watermark.
+ * @return This element as a Watermark.
+ * @throws java.lang.ClassCastException Thrown, if this element is actually not a Watermark.
+ */
+ public final Watermark asWatermark() {
+ return (Watermark) this;
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java
index 92ce66f..348b974 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java
@@ -22,10 +22,12 @@ package org.apache.flink.streaming.runtime.streamrecord;
*
* @param <T> The type encapsulated with the stream record.
*/
-public class StreamRecord<T> {
+public class StreamRecord<T> extends StreamElement {
+ /** The actual value held by this record */
private T value;
+ /** The timestamp of the record */
private long timestamp;
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
index b78ec44..1187fe6 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
@@ -20,7 +20,6 @@
package org.apache.flink.runtime.io.network.partition.consumer;
import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.HeapMemorySegment;
import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.event.AbstractEvent;
import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
@@ -32,6 +31,7 @@ import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
import org.apache.flink.runtime.plugable.SerializationDelegate;
import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
@@ -85,7 +85,8 @@ public class StreamTestSingleInputGate<T> extends TestSingleInputGate {
for (int i = 0; i < numInputChannels; i++) {
final int channelIndex = i;
final RecordSerializer<SerializationDelegate<Object>> recordSerializer = new SpanningRecordSerializer<SerializationDelegate<Object>>();
- final SerializationDelegate<Object> delegate = new SerializationDelegate<Object>(new MultiplexingStreamRecordSerializer<T>(serializer));
+ final SerializationDelegate<Object> delegate = (SerializationDelegate<Object>) (SerializationDelegate<?>)
+ new SerializationDelegate<StreamElement>(new MultiplexingStreamRecordSerializer<T>(serializer));
inputQueues[channelIndex] = new ConcurrentLinkedQueue<InputValue<Object>>();
inputChannels[channelIndex] = new TestInputChannel(inputGate, i);
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java
index bf4fe40..047aad8 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java
@@ -26,7 +26,6 @@ import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.joda.time.Instant;
import org.junit.Assert;
import org.junit.Test;
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
index cb08e65..dcfe3de 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
@@ -21,7 +21,6 @@ import java.util.concurrent.ConcurrentLinkedQueue;
import org.apache.flink.api.common.functions.FoldFunction;
import org.apache.flink.api.common.functions.RichFoldFunction;
-import org.apache.flink.api.common.functions.RichReduceFunction;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.functions.KeySelector;
@@ -31,7 +30,6 @@ import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.joda.time.Instant;
import org.junit.Assert;
import org.junit.Test;
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
index 9e35fa2..a2cd1fd 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
@@ -28,7 +28,6 @@ import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.joda.time.Instant;
import org.junit.Assert;
import org.junit.Test;
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java
index 4d12492..f0113d1 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java
@@ -26,7 +26,6 @@ import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.joda.time.Instant;
import org.junit.Assert;
import org.junit.Test;
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java
index bb9dad7..14abd18 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java
@@ -20,7 +20,6 @@ package org.apache.flink.streaming.api.operators;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
-import java.io.Serializable;
import java.util.HashSet;
import java.util.concurrent.ConcurrentLinkedQueue;
@@ -42,7 +41,7 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
+
import org.junit.Test;
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java
index 4c644a9..4dbf7b8 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java
@@ -23,19 +23,14 @@ import java.io.Serializable;
import java.util.concurrent.ConcurrentLinkedQueue;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.NoOpSink;
import org.apache.flink.streaming.util.TestHarnessUtil;
import org.apache.flink.streaming.util.TwoInputStreamOperatorTestHarness;
import org.apache.flink.util.Collector;
import org.junit.Assert;
-import org.junit.Ignore;
import org.junit.Test;
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
index 96dbeab..06fca6b 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
@@ -35,6 +35,7 @@ import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
import org.apache.flink.util.InstantiationUtil;
import org.junit.Assert;
@@ -79,7 +80,7 @@ public class StreamTaskTestHarness<OUT> {
private AbstractInvokable task;
private TypeSerializer<OUT> outputSerializer;
- private TypeSerializer<Object> outputStreamRecordSerializer;
+ private TypeSerializer<StreamElement> outputStreamRecordSerializer;
private ConcurrentLinkedQueue<Object> outputList;
@@ -119,8 +120,7 @@ public class StreamTaskTestHarness<OUT> {
/**
* This must be overwritten for OneInputStreamTask or TwoInputStreamTask test harnesses.
*/
- protected void initializeInputs() throws IOException, InterruptedException {
- }
+ protected void initializeInputs() throws IOException, InterruptedException {}
@SuppressWarnings("unchecked")
private void initializeOutput() {
[09/15] flink git commit: [FLINK-1320] [core] Add an off-heap variant
of the managed memory
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SimpleCollectingOutputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SimpleCollectingOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SimpleCollectingOutputView.java
index 35ec3e3..a107e79 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SimpleCollectingOutputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SimpleCollectingOutputView.java
@@ -25,7 +25,7 @@ import java.util.List;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.core.memory.MemorySegmentSource;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
import org.apache.flink.runtime.util.MathUtils;
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java
index 5f9c2cf..397c40c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java
@@ -30,7 +30,7 @@ import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader;
import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter;
import org.apache.flink.runtime.io.disk.iomanager.HeaderlessChannelReaderInputView;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
index b919034..de1ed87 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
@@ -25,7 +25,7 @@ import java.util.List;
import java.util.concurrent.LinkedBlockingQueue;
import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
/**
* A {@link org.apache.flink.core.memory.DataInputView} that is backed by a
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
index 089e10a..64b2ebc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
@@ -24,7 +24,7 @@ import java.util.List;
import java.util.concurrent.LinkedBlockingQueue;
import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
/**
* A {@link org.apache.flink.core.memory.DataOutputView} that is backed by a
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
index 9eb4a9a..6e4b564 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
@@ -107,7 +107,8 @@ public class NetworkEnvironment {
// create the network buffers - this is the operation most likely to fail upon
// mis-configuration, so we do this first
try {
- networkBufferPool = new NetworkBufferPool(config.numNetworkBuffers(), config.networkBufferSize());
+ networkBufferPool = new NetworkBufferPool(config.numNetworkBuffers(),
+ config.networkBufferSize(), config.memoryType());
}
catch (Throwable t) {
throw new IOException("Cannot allocate network buffer pool: " + t.getMessage(), t);
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java
index ec9f4fd..cdd8731 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java
@@ -230,21 +230,21 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
@Override
public final short readShort() throws IOException {
- final short v = this.segment.getShort(this.position);
+ final short v = this.segment.getShortBigEndian(this.position);
this.position += 2;
return v;
}
@Override
public final int readUnsignedShort() throws IOException {
- final int v = this.segment.getShort(this.position) & 0xffff;
+ final int v = this.segment.getShortBigEndian(this.position) & 0xffff;
this.position += 2;
return v;
}
@Override
public final char readChar() throws IOException {
- final char v = this.segment.getChar(this.position);
+ final char v = this.segment.getCharBigEndian(this.position);
this.position += 2;
return v;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java
index b23b83b..a34f8cf 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java
@@ -19,6 +19,7 @@
package org.apache.flink.runtime.io.network.api.serialization;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.event.AbstractEvent;
import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
@@ -145,8 +146,9 @@ public class EventSerializer {
public static Buffer toBuffer(AbstractEvent event) {
final ByteBuffer serializedEvent = EventSerializer.toSerializedEvent(event);
- final Buffer buffer = new Buffer(new MemorySegment(serializedEvent.array()),
- FreeingBufferRecycler.INSTANCE, false);
+ MemorySegment data = MemorySegmentFactory.wrap(serializedEvent.array());
+
+ final Buffer buffer = new Buffer(data, FreeingBufferRecycler.INSTANCE, false);
buffer.setSize(serializedEvent.remaining());
return buffer;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java
index f3e4892..b10e5a8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java
@@ -262,21 +262,21 @@ public class SpillingAdaptiveSpanningRecordDeserializer<T extends IOReadableWrit
@Override
public final short readShort() throws IOException {
- final short v = this.segment.getShort(this.position);
+ final short v = this.segment.getShortBigEndian(this.position);
this.position += 2;
return v;
}
@Override
public final int readUnsignedShort() throws IOException {
- final int v = this.segment.getShort(this.position) & 0xffff;
+ final int v = this.segment.getShortBigEndian(this.position) & 0xffff;
this.position += 2;
return v;
}
@Override
public final char readChar() throws IOException {
- final char v = this.segment.getChar(this.position);
+ final char v = this.segment.getCharBigEndian(this.position);
this.position += 2;
return v;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java
index 23c7ed0..5fb62cb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java
@@ -84,9 +84,8 @@ public class Buffer {
public ByteBuffer getNioBuffer() {
synchronized (recycleLock) {
ensureNotRecycled();
-
- // we need to return a copy here to guarantee thread-safety
- return memorySegment.wrap(0, currentSize).duplicate();
+ // the memory segment returns a distinct buffer every time
+ return memorySegment.wrap(0, currentSize);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java
index cb1f118..209d925 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java
@@ -20,15 +20,20 @@ package org.apache.flink.runtime.io.network.buffer;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.core.memory.MemoryType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
+import java.nio.ByteBuffer;
import java.util.HashSet;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ArrayBlockingQueue;
+import static com.google.common.base.Preconditions.checkNotNull;
+
/**
* The NetworkBufferPool is a fixed size pool of {@link MemorySegment} instances
* for the network stack.
@@ -62,7 +67,9 @@ public class NetworkBufferPool implements BufferPoolFactory {
/**
* Allocates all {@link MemorySegment} instances managed by this pool.
*/
- public NetworkBufferPool(int numberOfSegmentsToAllocate, int segmentSize) {
+ public NetworkBufferPool(int numberOfSegmentsToAllocate, int segmentSize, MemoryType memoryType) {
+ checkNotNull(memoryType);
+
this.totalNumberOfMemorySegments = numberOfSegmentsToAllocate;
this.memorySegmentSize = segmentSize;
@@ -72,12 +79,25 @@ public class NetworkBufferPool implements BufferPoolFactory {
this.availableMemorySegments = new ArrayBlockingQueue<MemorySegment>(numberOfSegmentsToAllocate);
}
catch (OutOfMemoryError err) {
- throw new OutOfMemoryError("Could not allocate buffer queue of length " + numberOfSegmentsToAllocate);
+ throw new OutOfMemoryError("Could not allocate buffer queue of length "
+ + numberOfSegmentsToAllocate + " - " + err.getMessage());
}
try {
- for (int i = 0; i < numberOfSegmentsToAllocate; i++) {
- availableMemorySegments.add(new MemorySegment(new byte[segmentSize]));
+ if (memoryType == MemoryType.HEAP) {
+ for (int i = 0; i < numberOfSegmentsToAllocate; i++) {
+ byte[] memory = new byte[segmentSize];
+ availableMemorySegments.add(MemorySegmentFactory.wrapPooledHeapMemory(memory, null));
+ }
+ }
+ else if (memoryType == MemoryType.OFF_HEAP) {
+ for (int i = 0; i < numberOfSegmentsToAllocate; i++) {
+ ByteBuffer memory = ByteBuffer.allocateDirect(segmentSize);
+ availableMemorySegments.add(MemorySegmentFactory.wrapPooledOffHeapMemory(memory, null));
+ }
+ }
+ else {
+ throw new IllegalArgumentException("Unknown memory type " + memoryType);
}
}
catch (OutOfMemoryError err) {
@@ -93,7 +113,7 @@ public class NetworkBufferPool implements BufferPoolFactory {
throw new OutOfMemoryError("Could not allocate enough memory segments for NetworkBufferPool " +
"(required (Mb): " + requiredMb +
", allocated (Mb): " + allocatedMb +
- ", missing (Mb): " + missingMb + ").");
+ ", missing (Mb): " + missingMb + "). Cause: " + err.getMessage());
}
long allocatedMb = (sizeInLong * availableMemorySegments.size()) >> 20;
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java
index 3b7d921..ee015c2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java
@@ -23,6 +23,7 @@ import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.buffer.BufferProvider;
import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
@@ -301,7 +302,8 @@ class PartitionRequestClientHandler extends ChannelInboundHandlerAdapter {
byte[] byteArray = new byte[bufferOrEvent.getSize()];
bufferOrEvent.getNettyBuffer().readBytes(byteArray);
- Buffer buffer = new Buffer(new MemorySegment(byteArray), FreeingBufferRecycler.INSTANCE, false);
+ MemorySegment memSeg = MemorySegmentFactory.wrap(byteArray);
+ Buffer buffer = new Buffer(memSeg, FreeingBufferRecycler.INSTANCE, false);
inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber);
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java
index 5b91668..2c0c744 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java
@@ -19,6 +19,7 @@
package org.apache.flink.runtime.io.network.partition;
import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.runtime.io.disk.iomanager.BufferFileReader;
import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
import org.apache.flink.runtime.io.disk.iomanager.SynchronousBufferFileReader;
@@ -136,7 +137,7 @@ class SpilledSubpartitionViewSyncIO implements ResultSubpartitionView {
synchronized (buffers) {
for (int i = 0; i < numberOfBuffers; i++) {
- buffers.add(new Buffer(new MemorySegment(new byte[memorySegmentSize]), this));
+ buffers.add(new Buffer(MemorySegmentFactory.allocateUnpooledSegment(memorySegmentSize), this));
}
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java
index ca1ade9..7776894 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java
@@ -33,8 +33,8 @@ import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader;
import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter;
import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
public class SerializedUpdateBuffer extends AbstractPagedOutputView {
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedInputView.java
new file mode 100644
index 0000000..788eebb
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedInputView.java
@@ -0,0 +1,568 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.runtime.memory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.MemorySegment;
+
+
+/**
+ * The base class for all input views that are backed by multiple memory pages. This base class contains all
+ * decoding methods to read data from a page and detect page boundary crossing. The concrete sub classes must
+ * implement the methods to provide the next memory page once the boundary is crossed.
+ */
+public abstract class AbstractPagedInputView implements DataInputView {
+
+ private MemorySegment currentSegment;
+
+ protected final int headerLength; // the number of bytes to skip at the beginning of each segment
+
+ private int positionInSegment; // the offset in the current segment
+
+ private int limitInSegment; // the limit in the current segment before switching to the next
+
+ private byte[] utfByteBuffer; // reusable byte buffer for utf-8 decoding
+ private char[] utfCharBuffer; // reusable char buffer for utf-8 decoding
+
+
+ // --------------------------------------------------------------------------------------------
+ // Constructors
+ // --------------------------------------------------------------------------------------------
+
+ /**
+ * Creates a new view that starts with the given segment. The input starts directly after the header
+ * of the given page. If the header size is zero, it starts at the beginning. The specified initial
+ * limit describes up to which position data may be read from the current segment, before the view must
+ * advance to the next segment.
+ *
+ * @param initialSegment The memory segment to start reading from.
+ * @param initialLimit The position one after the last valid byte in the initial segment.
+ * @param headerLength The number of bytes to skip at the beginning of each segment for the header. This
+ * length must be the same for all memory segments.
+ */
+ protected AbstractPagedInputView(MemorySegment initialSegment, int initialLimit, int headerLength) {
+ this.headerLength = headerLength;
+ this.positionInSegment = headerLength;
+ seekInput(initialSegment, headerLength, initialLimit);
+ }
+
+ /**
+ * Creates a new view that is initially not bound to a memory segment. This constructor is typically
+ * for views that always seek first.
+ * <p>
+ * WARNING: The view is not readable until the first call to either {@link #advance()},
+ * or to {@link #seekInput(MemorySegment, int, int)}.
+ *
+ * @param headerLength The number of bytes to skip at the beginning of each segment for the header.
+ */
+ protected AbstractPagedInputView(int headerLength) {
+ this.headerLength = headerLength;
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Page Management
+ // --------------------------------------------------------------------------------------------
+
+ /**
+ * Gets the memory segment that will be used to read the next bytes from. If the segment is exactly exhausted,
+ * meaning that the last byte read was the last byte available in the segment, then this segment will
+ * not serve the next bytes. The segment to serve the next bytes will be obtained through the
+ * {@link #nextSegment(MemorySegment)} method.
+ *
+ * @return The current memory segment.
+ */
+ public MemorySegment getCurrentSegment() {
+ return this.currentSegment;
+ }
+
+ /**
+ * Gets the position from which the next byte will be read. If that position is equal to the current limit,
+ * then the next byte will be read from next segment.
+ *
+ * @return The position from which the next byte will be read.
+ * @see #getCurrentSegmentLimit()
+ */
+ public int getCurrentPositionInSegment() {
+ return this.positionInSegment;
+ }
+
+ /**
+ * Gets the current limit in the memory segment. This value points to the byte one after the last valid byte
+ * in the memory segment.
+ *
+ * @return The current limit in the memory segment.
+ * @see #getCurrentPositionInSegment()
+ */
+ public int getCurrentSegmentLimit() {
+ return this.limitInSegment;
+ }
+
+ /**
+ * The method by which concrete subclasses realize page crossing. This method is invoked when the current page
+ * is exhausted and a new page is required to continue the reading. If no further page is available, this
+ * method must throw an {@link EOFException}.
+ *
+ * @param current The current page that was read to its limit. May be {@code null}, if this method is
+ * invoked for the first time.
+ * @return The next page from which the reading should continue. May not be {@code null}. If the input is
+ * exhausted, an {@link EOFException} must be thrown instead.
+ *
+ * @throws EOFException Thrown, if no further segment is available.
+ * @throws IOException Thrown, if the method cannot provide the next page due to an I/O related problem.
+ */
+ protected abstract MemorySegment nextSegment(MemorySegment current) throws EOFException, IOException;
+
+ /**
+ * Gets the limit for reading bytes from the given memory segment. This method must return the position
+ * of the byte after the last valid byte in the given memory segment. When the position returned by this
+ * method is reached, the view will attempt to switch to the next memory segment.
+ *
+ * @param segment The segment to determine the limit for.
+ * @return The limit for the given memory segment.
+ */
+ protected abstract int getLimitForSegment(MemorySegment segment);
+
+ /**
+ * Advances the view to the next memory segment. The reading will continue after the header of the next
+ * segment. This method uses {@link #nextSegment(MemorySegment)} and {@link #getLimitForSegment(MemorySegment)}
+ * to get the next segment and set its limit.
+ *
+ * @throws IOException Thrown, if the next segment could not be obtained.
+ *
+ * @see #nextSegment(MemorySegment)
+ * @see #getLimitForSegment(MemorySegment)
+ */
+ protected final void advance() throws IOException {
+ // note: this code ensures that in case of EOF, we stay at the same position such that
+ // EOF is reproducible (if nextSegment throws a reproducible EOFException)
+ this.currentSegment = nextSegment(this.currentSegment);
+ this.limitInSegment = getLimitForSegment(this.currentSegment);
+ this.positionInSegment = this.headerLength;
+ }
+
+ /**
+ * Sets the internal state of the view such that the next bytes will be read from the given memory segment,
+ * starting at the given position. The memory segment will provide bytes up to the given limit position.
+ *
+ * @param segment The segment to read the next bytes from.
+ * @param positionInSegment The position in the segment to start reading from.
+ * @param limitInSegment The limit in the segment. When reached, the view will attempt to switch to
+ * the next segment.
+ */
+ protected void seekInput(MemorySegment segment, int positionInSegment, int limitInSegment) {
+ this.currentSegment = segment;
+ this.positionInSegment = positionInSegment;
+ this.limitInSegment = limitInSegment;
+ }
+
+ /**
+ * Clears the internal state of the view. After this call, all read attempts will fail, until the
+ * {@link #advance()} or {@link #seekInput(MemorySegment, int, int)} method have been invoked.
+ */
+ protected void clear() {
+ this.currentSegment = null;
+ this.positionInSegment = this.headerLength;
+ this.limitInSegment = headerLength;
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Data Input Specific methods
+ // --------------------------------------------------------------------------------------------
+
+ @Override
+ public int read(byte[] b) throws IOException{
+ return read(b,0,b.length);
+ }
+
+ @Override
+ public int read(byte[] b, int off, int len) throws IOException{
+ if (off < 0 || len < 0 || off + len > b.length) {
+ throw new IndexOutOfBoundsException();
+ }
+
+ int remaining = this.limitInSegment - this.positionInSegment;
+ if (remaining >= len) {
+ this.currentSegment.get(this.positionInSegment, b, off, len);
+ this.positionInSegment += len;
+ return len;
+ }
+ else {
+ if (remaining == 0) {
+ try {
+ advance();
+ }
+ catch (EOFException eof) {
+ return -1;
+ }
+ remaining = this.limitInSegment - this.positionInSegment;
+ }
+
+ int bytesRead = 0;
+ while (true) {
+ int toRead = Math.min(remaining, len-bytesRead);
+ this.currentSegment.get(this.positionInSegment, b, off, toRead);
+ off += toRead;
+ bytesRead += toRead;
+
+ if (len > bytesRead) {
+ try {
+ advance();
+ }
+ catch (EOFException eof) {
+ this.positionInSegment += toRead;
+ return bytesRead;
+ }
+ remaining = this.limitInSegment - this.positionInSegment;
+ }
+ else {
+ this.positionInSegment += toRead;
+ break;
+ }
+ }
+ return len;
+ }
+ }
+
+ @Override
+ public void readFully(byte[] b) throws IOException {
+ readFully(b, 0, b.length);
+ }
+
+ @Override
+ public void readFully(byte[] b, int off, int len) throws IOException {
+ int bytesRead = read(b,off,len);
+
+ if(bytesRead < len){
+ throw new EOFException("There is no enough data left in the DataInputView.");
+ }
+ }
+
+ @Override
+ public boolean readBoolean() throws IOException {
+ return readByte() == 1;
+ }
+
+ @Override
+ public byte readByte() throws IOException {
+ if (this.positionInSegment < this.limitInSegment) {
+ return this.currentSegment.get(this.positionInSegment++);
+ }
+ else {
+ advance();
+ return readByte();
+ }
+ }
+
+ @Override
+ public int readUnsignedByte() throws IOException {
+ return readByte() & 0xff;
+ }
+
+ @Override
+ public short readShort() throws IOException {
+ if (this.positionInSegment < this.limitInSegment - 1) {
+ final short v = this.currentSegment.getShortBigEndian(this.positionInSegment);
+ this.positionInSegment += 2;
+ return v;
+ }
+ else if (this.positionInSegment == this.limitInSegment) {
+ advance();
+ return readShort();
+ }
+ else {
+ return (short) ((readUnsignedByte() << 8) | readUnsignedByte());
+ }
+ }
+
+ @Override
+ public int readUnsignedShort() throws IOException {
+ if (this.positionInSegment < this.limitInSegment - 1) {
+ final int v = this.currentSegment.getShortBigEndian(this.positionInSegment) & 0xffff;
+ this.positionInSegment += 2;
+ return v;
+ }
+ else if (this.positionInSegment == this.limitInSegment) {
+ advance();
+ return readUnsignedShort();
+ }
+ else {
+ return (readUnsignedByte() << 8) | readUnsignedByte();
+ }
+ }
+
+ @Override
+ public char readChar() throws IOException {
+ if (this.positionInSegment < this.limitInSegment - 1) {
+ final char v = this.currentSegment.getCharBigEndian(this.positionInSegment);
+ this.positionInSegment += 2;
+ return v;
+ }
+ else if (this.positionInSegment == this.limitInSegment) {
+ advance();
+ return readChar();
+ }
+ else {
+ return (char) ((readUnsignedByte() << 8) | readUnsignedByte());
+ }
+ }
+
+ @Override
+ public int readInt() throws IOException {
+ if (this.positionInSegment < this.limitInSegment - 3) {
+ final int v = this.currentSegment.getIntBigEndian(this.positionInSegment);
+ this.positionInSegment += 4;
+ return v;
+ }
+ else if (this.positionInSegment == this.limitInSegment) {
+ advance();
+ return readInt();
+ }
+ else {
+ return (readUnsignedByte() << 24) |
+ (readUnsignedByte() << 16) |
+ (readUnsignedByte() << 8) |
+ readUnsignedByte();
+ }
+ }
+
+ @Override
+ public long readLong() throws IOException {
+ if (this.positionInSegment < this.limitInSegment - 7) {
+ final long v = this.currentSegment.getLongBigEndian(this.positionInSegment);
+ this.positionInSegment += 8;
+ return v;
+ }
+ else if (this.positionInSegment == this.limitInSegment) {
+ advance();
+ return readLong();
+ }
+ else {
+ long l = 0L;
+ l |= ((long) readUnsignedByte()) << 56;
+ l |= ((long) readUnsignedByte()) << 48;
+ l |= ((long) readUnsignedByte()) << 40;
+ l |= ((long) readUnsignedByte()) << 32;
+ l |= ((long) readUnsignedByte()) << 24;
+ l |= ((long) readUnsignedByte()) << 16;
+ l |= ((long) readUnsignedByte()) << 8;
+ l |= (long) readUnsignedByte();
+ return l;
+ }
+ }
+
+ @Override
+ public float readFloat() throws IOException {
+ return Float.intBitsToFloat(readInt());
+ }
+
+ @Override
+ public double readDouble() throws IOException {
+ return Double.longBitsToDouble(readLong());
+ }
+
+ @Override
+ public String readLine() throws IOException {
+ final StringBuilder bld = new StringBuilder(32);
+
+ try {
+ int b;
+ while ((b = readUnsignedByte()) != '\n') {
+ if (b != '\r') {
+ bld.append((char) b);
+ }
+ }
+ }
+ catch (EOFException eofex) {}
+
+ if (bld.length() == 0) {
+ return null;
+ }
+
+ // trim a trailing carriage return
+ int len = bld.length();
+ if (len > 0 && bld.charAt(len - 1) == '\r') {
+ bld.setLength(len - 1);
+ }
+ return bld.toString();
+ }
+
+ @Override
+ public String readUTF() throws IOException {
+ final int utflen = readUnsignedShort();
+
+ final byte[] bytearr;
+ final char[] chararr;
+
+ if (this.utfByteBuffer == null || this.utfByteBuffer.length < utflen) {
+ bytearr = new byte[utflen];
+ this.utfByteBuffer = bytearr;
+ } else {
+ bytearr = this.utfByteBuffer;
+ }
+ if (this.utfCharBuffer == null || this.utfCharBuffer.length < utflen) {
+ chararr = new char[utflen];
+ this.utfCharBuffer = chararr;
+ } else {
+ chararr = this.utfCharBuffer;
+ }
+
+ int c, char2, char3;
+ int count = 0;
+ int chararr_count = 0;
+
+ readFully(bytearr, 0, utflen);
+
+ while (count < utflen) {
+ c = (int) bytearr[count] & 0xff;
+ if (c > 127) {
+ break;
+ }
+ count++;
+ chararr[chararr_count++] = (char) c;
+ }
+
+ while (count < utflen) {
+ c = (int) bytearr[count] & 0xff;
+ switch (c >> 4) {
+ case 0:
+ case 1:
+ case 2:
+ case 3:
+ case 4:
+ case 5:
+ case 6:
+ case 7:
+ /* 0xxxxxxx */
+ count++;
+ chararr[chararr_count++] = (char) c;
+ break;
+ case 12:
+ case 13:
+ /* 110x xxxx 10xx xxxx */
+ count += 2;
+ if (count > utflen) {
+ throw new UTFDataFormatException("malformed input: partial character at end");
+ }
+ char2 = (int) bytearr[count - 1];
+ if ((char2 & 0xC0) != 0x80) {
+ throw new UTFDataFormatException("malformed input around byte " + count);
+ }
+ chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
+ break;
+ case 14:
+ /* 1110 xxxx 10xx xxxx 10xx xxxx */
+ count += 3;
+ if (count > utflen) {
+ throw new UTFDataFormatException("malformed input: partial character at end");
+ }
+ char2 = (int) bytearr[count - 2];
+ char3 = (int) bytearr[count - 1];
+ if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
+ throw new UTFDataFormatException("malformed input around byte " + (count - 1));
+ }
+ chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
+ break;
+ default:
+ /* 10xx xxxx, 1111 xxxx */
+ throw new UTFDataFormatException("malformed input around byte " + count);
+ }
+ }
+ // The number of chars produced may be less than utflen
+ return new String(chararr, 0, chararr_count);
+ }
+
+ @Override
+ public int skipBytes(int n) throws IOException {
+ if (n < 0) {
+ throw new IllegalArgumentException();
+ }
+
+ int remaining = this.limitInSegment - this.positionInSegment;
+ if (remaining >= n) {
+ this.positionInSegment += n;
+ return n;
+ }
+ else {
+ if (remaining == 0) {
+ try {
+ advance();
+ } catch (EOFException eofex) {
+ return 0;
+ }
+ remaining = this.limitInSegment - this.positionInSegment;
+ }
+
+ int skipped = 0;
+ while (true) {
+ int toSkip = Math.min(remaining, n);
+ n -= toSkip;
+ skipped += toSkip;
+
+ if (n > 0) {
+ try {
+ advance();
+ } catch (EOFException eofex) {
+ return skipped;
+ }
+ remaining = this.limitInSegment - this.positionInSegment;
+ }
+ else {
+ this.positionInSegment += toSkip;
+ break;
+ }
+ }
+ return skipped;
+ }
+ }
+
+ @Override
+ public void skipBytesToRead(int numBytes) throws IOException {
+ if (numBytes < 0) {
+ throw new IllegalArgumentException();
+ }
+
+ int remaining = this.limitInSegment - this.positionInSegment;
+ if (remaining >= numBytes) {
+ this.positionInSegment += numBytes;
+ }
+ else {
+ if (remaining == 0) {
+ advance();
+ remaining = this.limitInSegment - this.positionInSegment;
+ }
+
+ while (true) {
+ if (numBytes > remaining) {
+ numBytes -= remaining;
+ advance();
+ remaining = this.limitInSegment - this.positionInSegment;
+ }
+ else {
+ this.positionInSegment += numBytes;
+ break;
+ }
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedOutputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedOutputView.java
new file mode 100644
index 0000000..482d82e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedOutputView.java
@@ -0,0 +1,414 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.runtime.memory;
+
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+
+
+/**
+ * The base class for all output views that are backed by multiple memory pages. This base class contains all
+ * encoding methods to write data to a page and detect page boundary crossing. The concrete sub classes must
+ * implement the methods to collect the current page and provide the next memory page once the boundary is crossed.
+ * <p>
+ * The paging assumes that all memory segments are of the same size.
+ */
+public abstract class AbstractPagedOutputView implements DataOutputView {
+
+ private MemorySegment currentSegment; // the current memory segment to write to
+
+ protected final int segmentSize; // the size of the memory segments
+
+ protected final int headerLength; // the number of bytes to skip at the beginning of each segment
+
+ private int positionInSegment; // the offset in the current segment
+
+ private byte[] utfBuffer; // the reusable array for UTF encodings
+
+
+ // --------------------------------------------------------------------------------------------
+ // Constructors
+ // --------------------------------------------------------------------------------------------
+
+ /**
+ * Creates a new output view that writes initially to the given initial segment. All segments in the
+ * view have to be of the given {@code segmentSize}. A header of length {@code headerLength} is left
+ * at the beginning of each segment.
+ *
+ * @param initialSegment The segment that the view starts writing to.
+ * @param segmentSize The size of the memory segments.
+ * @param headerLength The number of bytes to skip at the beginning of each segment for the header.
+ */
+ protected AbstractPagedOutputView(MemorySegment initialSegment, int segmentSize, int headerLength) {
+ if (initialSegment == null) {
+ throw new NullPointerException("Initial Segment may not be null");
+ }
+ this.segmentSize = segmentSize;
+ this.headerLength = headerLength;
+ this.currentSegment = initialSegment;
+ this.positionInSegment = headerLength;
+ }
+
+ /**
+ * @param segmentSize The size of the memory segments.
+ * @param headerLength The number of bytes to skip at the beginning of each segment for the header.
+ */
+ protected AbstractPagedOutputView(int segmentSize, int headerLength)
+ {
+ this.segmentSize = segmentSize;
+ this.headerLength = headerLength;
+ }
+
+
+ // --------------------------------------------------------------------------------------------
+ // Page Management
+ // --------------------------------------------------------------------------------------------
+
+ /**
+ *
+ * This method must return a segment. If no more segments are available, it must throw an
+ * {@link java.io.EOFException}.
+ *
+ * @param current The current memory segment
+ * @param positionInCurrent The position in the segment, one after the last valid byte.
+ * @return The next memory segment.
+ *
+ * @throws IOException
+ */
+ protected abstract MemorySegment nextSegment(MemorySegment current, int positionInCurrent) throws IOException;
+
+
+ /**
+ * Gets the segment that the view currently writes to.
+ *
+ * @return The segment the view currently writes to.
+ */
+ public MemorySegment getCurrentSegment() {
+ return this.currentSegment;
+ }
+
+ /**
+ * Gets the current write position (the position where the next bytes will be written)
+ * in the current memory segment.
+ *
+ * @return The current write offset in the current memory segment.
+ */
+ public int getCurrentPositionInSegment() {
+ return this.positionInSegment;
+ }
+
+ /**
+ * Gets the size of the segments used by this view.
+ *
+ * @return The memory segment size.
+ */
+ public int getSegmentSize() {
+ return this.segmentSize;
+ }
+
+ /**
+ * Moves the output view to the next page. This method invokes internally the
+ * {@link #nextSegment(MemorySegment, int)} method to give the current memory segment to the concrete subclass'
+ * implementation and obtain the next segment to write to. Writing will continue inside the new segment
+ * after the header.
+ *
+ * @throws IOException Thrown, if the current segment could not be processed or a new segment could not
+ * be obtained.
+ */
+ protected void advance() throws IOException {
+ this.currentSegment = nextSegment(this.currentSegment, this.positionInSegment);
+ this.positionInSegment = this.headerLength;
+ }
+
+ /**
+ * Sets the internal state to the given memory segment and the given position within the segment.
+ *
+ * @param seg The memory segment to write the next bytes to.
+ * @param position The position to start writing the next bytes to.
+ */
+ protected void seekOutput(MemorySegment seg, int position) {
+ this.currentSegment = seg;
+ this.positionInSegment = position;
+ }
+
+ /**
+ * Clears the internal state. Any successive write calls will fail until either {@link #advance()} or
+ * {@link #seekOutput(MemorySegment, int)} is called.
+ *
+ * @see #advance()
+ * @see #seekOutput(MemorySegment, int)
+ */
+ protected void clear() {
+ this.currentSegment = null;
+ this.positionInSegment = this.headerLength;
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Data Output Specific methods
+ // --------------------------------------------------------------------------------------------
+
+ @Override
+ public void write(int b) throws IOException {
+ writeByte(b);
+ }
+
+ @Override
+ public void write(byte[] b) throws IOException {
+ write(b, 0, b.length);
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ int remaining = this.segmentSize - this.positionInSegment;
+ if (remaining >= len) {
+ this.currentSegment.put(this.positionInSegment, b, off, len);
+ this.positionInSegment += len;
+ }
+ else {
+ if (remaining == 0) {
+ advance();
+ remaining = this.segmentSize - this.positionInSegment;
+ }
+ while (true) {
+ int toPut = Math.min(remaining, len);
+ this.currentSegment.put(this.positionInSegment, b, off, toPut);
+ off += toPut;
+ len -= toPut;
+
+ if (len > 0) {
+ this.positionInSegment = this.segmentSize;
+ advance();
+ remaining = this.segmentSize - this.positionInSegment;
+ }
+ else {
+ this.positionInSegment += toPut;
+ break;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void writeBoolean(boolean v) throws IOException {
+ writeByte(v ? 1 : 0);
+ }
+
+ @Override
+ public void writeByte(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize) {
+ this.currentSegment.put(this.positionInSegment++, (byte) v);
+ }
+ else {
+ advance();
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeShort(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 1) {
+ this.currentSegment.putShortBigEndian(this.positionInSegment, (short) v);
+ this.positionInSegment += 2;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeShort(v);
+ }
+ else {
+ writeByte(v >> 8);
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeChar(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 1) {
+ this.currentSegment.putCharBigEndian(this.positionInSegment, (char) v);
+ this.positionInSegment += 2;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeChar(v);
+ }
+ else {
+ writeByte(v >> 8);
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeInt(int v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 3) {
+ this.currentSegment.putIntBigEndian(this.positionInSegment, v);
+ this.positionInSegment += 4;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeInt(v);
+ }
+ else {
+ writeByte(v >> 24);
+ writeByte(v >> 16);
+ writeByte(v >> 8);
+ writeByte(v);
+ }
+ }
+
+ @Override
+ public void writeLong(long v) throws IOException {
+ if (this.positionInSegment < this.segmentSize - 7) {
+ this.currentSegment.putLongBigEndian(this.positionInSegment, v);
+ this.positionInSegment += 8;
+ }
+ else if (this.positionInSegment == this.segmentSize) {
+ advance();
+ writeLong(v);
+ }
+ else {
+ writeByte((int) (v >> 56));
+ writeByte((int) (v >> 48));
+ writeByte((int) (v >> 40));
+ writeByte((int) (v >> 32));
+ writeByte((int) (v >> 24));
+ writeByte((int) (v >> 16));
+ writeByte((int) (v >> 8));
+ writeByte((int) v);
+ }
+ }
+
+ @Override
+ public void writeFloat(float v) throws IOException {
+ writeInt(Float.floatToRawIntBits(v));
+ }
+
+ @Override
+ public void writeDouble(double v) throws IOException {
+ writeLong(Double.doubleToRawLongBits(v));
+ }
+
+ @Override
+ public void writeBytes(String s) throws IOException {
+ for (int i = 0; i < s.length(); i++) {
+ writeByte(s.charAt(i));
+ }
+ }
+
+ @Override
+ public void writeChars(String s) throws IOException {
+ for (int i = 0; i < s.length(); i++) {
+ writeChar(s.charAt(i));
+ }
+ }
+
+ @Override
+ public void writeUTF(String str) throws IOException {
+ int strlen = str.length();
+ int utflen = 0;
+ int c, count = 0;
+
+ /* use charAt instead of copying String to char array */
+ for (int i = 0; i < strlen; i++) {
+ c = str.charAt(i);
+ if ((c >= 0x0001) && (c <= 0x007F)) {
+ utflen++;
+ } else if (c > 0x07FF) {
+ utflen += 3;
+ } else {
+ utflen += 2;
+ }
+ }
+
+ if (utflen > 65535) {
+ throw new UTFDataFormatException("encoded string too long: " + utflen + " memory");
+ }
+
+ if (this.utfBuffer == null || this.utfBuffer.length < utflen + 2) {
+ this.utfBuffer = new byte[utflen + 2];
+ }
+ final byte[] bytearr = this.utfBuffer;
+
+ bytearr[count++] = (byte) ((utflen >>> 8) & 0xFF);
+ bytearr[count++] = (byte) ( utflen & 0xFF);
+
+ int i;
+ for (i = 0; i < strlen; i++) {
+ c = str.charAt(i);
+ if (!((c >= 0x0001) && (c <= 0x007F))) {
+ break;
+ }
+ bytearr[count++] = (byte) c;
+ }
+
+ for (; i < strlen; i++) {
+ c = str.charAt(i);
+ if ((c >= 0x0001) && (c <= 0x007F)) {
+ bytearr[count++] = (byte) c;
+
+ } else if (c > 0x07FF) {
+ bytearr[count++] = (byte) (0xE0 | ((c >> 12) & 0x0F));
+ bytearr[count++] = (byte) (0x80 | ((c >> 6) & 0x3F));
+ bytearr[count++] = (byte) (0x80 | ( c & 0x3F));
+ } else {
+ bytearr[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F));
+ bytearr[count++] = (byte) (0x80 | ( c & 0x3F));
+ }
+ }
+
+ write(bytearr, 0, utflen + 2);
+ }
+
+ @Override
+ public void skipBytesToWrite(int numBytes) throws IOException {
+ while (numBytes > 0) {
+ final int remaining = this.segmentSize - this.positionInSegment;
+ if (numBytes <= remaining) {
+ this.positionInSegment += numBytes;
+ return;
+ }
+ this.positionInSegment = this.segmentSize;
+ advance();
+ numBytes -= remaining;
+ }
+ }
+
+ @Override
+ public void write(DataInputView source, int numBytes) throws IOException {
+ while (numBytes > 0) {
+ final int remaining = this.segmentSize - this.positionInSegment;
+ if (numBytes <= remaining) {
+ this.currentSegment.put(source, this.positionInSegment, numBytes);
+ this.positionInSegment += numBytes;
+ return;
+ }
+
+ if (remaining > 0) {
+ this.currentSegment.put(source, this.positionInSegment, remaining);
+ this.positionInSegment = this.segmentSize;
+ numBytes -= remaining;
+ }
+
+ advance();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memory/ListMemorySegmentSource.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/ListMemorySegmentSource.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/ListMemorySegmentSource.java
new file mode 100644
index 0000000..ebeb47d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/ListMemorySegmentSource.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.runtime.memory;
+
+import java.util.List;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentSource;
+
+/**
+ * Simple memory segment source that draws segments from a list.
+ *
+ */
+public class ListMemorySegmentSource implements MemorySegmentSource
+{
+ private final List<MemorySegment> segments;
+
+ public ListMemorySegmentSource(final List<MemorySegment> memorySegments) {
+ this.segments = memorySegments;
+ }
+
+
+ @Override
+ public MemorySegment nextSegment() {
+ if (this.segments.size() > 0) {
+ return this.segments.remove(this.segments.size() - 1);
+ } else {
+ return null;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryAllocationException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryAllocationException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryAllocationException.java
new file mode 100644
index 0000000..112d6f7
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryAllocationException.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.runtime.memory;
+
+/**
+ * An exception to be thrown when a memory allocation operation is not successful.
+ */
+public class MemoryAllocationException extends Exception {
+
+ private static final long serialVersionUID = -403983866457947012L;
+
+ public MemoryAllocationException() {
+ super();
+ }
+
+ public MemoryAllocationException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ public MemoryAllocationException(String message) {
+ super(message);
+ }
+
+ public MemoryAllocationException(Throwable cause) {
+ super(cause);
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java
new file mode 100644
index 0000000..094d065
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java
@@ -0,0 +1,700 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.memory;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.ConcurrentModificationException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.core.memory.HybridMemorySegment;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
+import org.apache.flink.runtime.util.MathUtils;
+
+/**
+ * The memory manager governs the memory that Flink uses for sorting, hashing, and caching. Memory
+ * is represented in segments of equal size. Operators allocate the memory by requesting a number
+ * of memory segments.
+ * <p>
+ * The memory may be represented as on-heap byte arrays ({@link HeapMemorySegment}), or as off-heap
+ * memory regions ({@link HybridMemorySegment}). Which kind of memory the MemoryManager serves can
+ * be passed as an argument to the initialization.
+ * <p>
+ * The memory manager can either pre-allocate all memory, or allocate the memory on demand. In the
+ * former version, memory will be occupied and reserved from start on, which means that no OutOfMemoryError
+ * can come while requesting memory. Released memory will also return to the MemoryManager's pool.
+ * On-demand allocation means that the memory manager only keeps track how many memory segments are
+ * currently allocated (bookkeeping only). Releasing a memory segment will not add it back to the pool,
+ * but make it re-claimable by the garbage collector.
+ */
+public class MemoryManager {
+
+ /** The default memory page size. Currently set to 32 KiBytes. */
+ public static final int DEFAULT_PAGE_SIZE = 32 * 1024;
+
+ /** The minimal memory page size. Currently set to 4 KiBytes. */
+ public static final int MIN_PAGE_SIZE = 4 * 1024;
+
+ // ------------------------------------------------------------------------
+
+ /** The lock used on the shared structures. */
+ private final Object lock = new Object();
+
+ /** The memory pool from which we draw memory segments. Specific to on-heap or off-heap memory */
+ private final MemoryPool memoryPool;
+
+ /** Memory segments allocated per memory owner */
+ private final HashMap<Object, Set<MemorySegment>> allocatedSegments;
+
+ /** The type of memory governed by this memory manager */
+ private final MemoryType memoryType;
+
+ /** mask used to round down sizes to multiples of the page size */
+ private final long roundingMask;
+
+ /** The size of the memory segments */
+ private final int pageSize;
+
+ /** The initial total size, for verification. */
+ private final int totalNumPages;
+
+ /** The total size of the memory managed by this memory manager */
+ private final long memorySize;
+
+ /** Number of slots of the task manager */
+ private final int numberOfSlots;
+
+ /** Flag marking whether the memory manager immediately allocates the memory */
+ private final boolean isPreAllocated;
+
+ /** The number of memory pages that have not been allocated and are available for lazy allocation */
+ private int numNonAllocatedPages;
+
+ /** flag whether the close() has already been invoked */
+ private boolean isShutDown;
+
+
+ /**
+ * Creates a memory manager with the given capacity, using the default page size.
+ *
+ * @param memorySize The total size of the memory to be managed by this memory manager.
+ * @param numberOfSlots The number of slots of the task manager.
+ */
+ public MemoryManager(long memorySize, int numberOfSlots) {
+ this(memorySize, numberOfSlots, DEFAULT_PAGE_SIZE, MemoryType.HEAP, true);
+ }
+
+ /**
+ * Creates a memory manager with the given capacity and given page size.
+ *
+ * @param memorySize The total size of the memory to be managed by this memory manager.
+ * @param numberOfSlots The number of slots of the task manager.
+ * @param pageSize The size of the pages handed out by the memory manager.
+ * @param memoryType The type of memory (heap / off-heap) that the memory manager should allocate.
+ * @param preAllocateMemory True, if the memory manager should immediately allocate all memory, false
+ * if it should allocate and release the memory as needed.
+ */
+ public MemoryManager(long memorySize, int numberOfSlots, int pageSize,
+ MemoryType memoryType, boolean preAllocateMemory) {
+ // sanity checks
+ if (memoryType == null) {
+ throw new NullPointerException();
+ }
+ if (memorySize <= 0) {
+ throw new IllegalArgumentException("Size of total memory must be positive.");
+ }
+ if (pageSize < MIN_PAGE_SIZE) {
+ throw new IllegalArgumentException("The page size must be at least " + MIN_PAGE_SIZE + " bytes.");
+ }
+ if (!MathUtils.isPowerOf2(pageSize)) {
+ throw new IllegalArgumentException("The given page size is not a power of two.");
+ }
+
+ this.memoryType = memoryType;
+ this.memorySize = memorySize;
+ this.numberOfSlots = numberOfSlots;
+
+ // assign page size and bit utilities
+ this.pageSize = pageSize;
+ this.roundingMask = ~((long) (pageSize - 1));
+
+ final long numPagesLong = memorySize / pageSize;
+ if (numPagesLong > Integer.MAX_VALUE) {
+ throw new IllegalArgumentException("The given number of memory bytes (" + memorySize
+ + ") corresponds to more than MAX_INT pages.");
+ }
+ this.totalNumPages = (int) numPagesLong;
+ if (this.totalNumPages < 1) {
+ throw new IllegalArgumentException("The given amount of memory amounted to less than one page.");
+ }
+
+ this.allocatedSegments = new HashMap<Object, Set<MemorySegment>>();
+ this.isPreAllocated = preAllocateMemory;
+
+ this.numNonAllocatedPages = preAllocateMemory ? 0 : this.totalNumPages;
+ final int memToAllocate = preAllocateMemory ? this.totalNumPages : 0;
+
+ switch (memoryType) {
+ case HEAP:
+ this.memoryPool = new HeapMemoryPool(memToAllocate, pageSize);
+ break;
+ case OFF_HEAP:
+ this.memoryPool = new HybridOffHeapMemoryPool(memToAllocate, pageSize);
+ break;
+ default:
+ throw new IllegalArgumentException("unrecognized memory type: " + memoryType);
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Shutdown
+ // ------------------------------------------------------------------------
+
+ /**
+ * Shuts the memory manager down, trying to release all the memory it managed. Depending
+ * on implementation details, the memory does not necessarily become reclaimable by the
+ * garbage collector, because there might still be references to allocated segments in the
+ * code that allocated them from the memory manager.
+ */
+ public void shutdown() {
+ // -------------------- BEGIN CRITICAL SECTION -------------------
+ synchronized (lock)
+ {
+ if (!isShutDown) {
+ // mark as shutdown and release memory
+ isShutDown = true;
+ numNonAllocatedPages = 0;
+
+ // go over all allocated segments and release them
+ for (Set<MemorySegment> segments : allocatedSegments.values()) {
+ for (MemorySegment seg : segments) {
+ seg.free();
+ }
+ }
+
+ memoryPool.clear();
+ }
+ }
+ // -------------------- END CRITICAL SECTION -------------------
+ }
+
+ /**
+ * Checks whether the MemoryManager has been shut down.
+ *
+ * @return True, if the memory manager is shut down, false otherwise.
+ */
+ public boolean isShutdown() {
+ return isShutDown;
+ }
+
+ /**
+ * Checks if the memory manager all memory available.
+ *
+ * @return True, if the memory manager is empty and valid, false if it is not empty or corrupted.
+ */
+ public boolean verifyEmpty() {
+ synchronized (lock) {
+ return isPreAllocated ?
+ memoryPool.getNumberOfAvailableMemorySegments() == totalNumPages :
+ numNonAllocatedPages == totalNumPages;
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Memory allocation and release
+ // ------------------------------------------------------------------------
+
+ /**
+ * Allocates a set of memory segments from this memory manager. If the memory manager pre-allocated the
+ * segments, they will be taken from the pool of memory segments. Otherwise, they will be allocated
+ * as part of this call.
+ *
+ * @param owner The owner to associate with the memory segment, for the fallback release.
+ * @param numPages The number of pages to allocate.
+ * @return A list with the memory segments.
+ * @throws MemoryAllocationException Thrown, if this memory manager does not have the requested amount
+ * of memory pages any more.
+ */
+ public List<MemorySegment> allocatePages(Object owner, int numPages) throws MemoryAllocationException {
+ final ArrayList<MemorySegment> segs = new ArrayList<MemorySegment>(numPages);
+ allocatePages(owner, segs, numPages);
+ return segs;
+ }
+
+ /**
+ * Allocates a set of memory segments from this memory manager. If the memory manager pre-allocated the
+ * segments, they will be taken from the pool of memory segments. Otherwise, they will be allocated
+ * as part of this call.
+ *
+ * @param owner The owner to associate with the memory segment, for the fallback release.
+ * @param target The list into which to put the allocated memory pages.
+ * @param numPages The number of pages to allocate.
+ * @throws MemoryAllocationException Thrown, if this memory manager does not have the requested amount
+ * of memory pages any more.
+ */
+ public void allocatePages(Object owner, List<MemorySegment> target, int numPages)
+ throws MemoryAllocationException
+ {
+ // sanity check
+ if (owner == null) {
+ throw new IllegalArgumentException("The memory owner must not be null.");
+ }
+
+ // reserve array space, if applicable
+ if (target instanceof ArrayList) {
+ ((ArrayList<MemorySegment>) target).ensureCapacity(numPages);
+ }
+
+ // -------------------- BEGIN CRITICAL SECTION -------------------
+ synchronized (lock)
+ {
+ if (isShutDown) {
+ throw new IllegalStateException("Memory manager has been shut down.");
+ }
+
+ // in the case of pre-allocated memory, the 'numNonAllocatedPages' is zero, in the
+ // lazy case, the 'freeSegments.size()' is zero.
+ if (numPages > (memoryPool.getNumberOfAvailableMemorySegments() + numNonAllocatedPages)) {
+ throw new MemoryAllocationException("Could not allocate " + numPages + " pages. Only " +
+ (memoryPool.getNumberOfAvailableMemorySegments() + numNonAllocatedPages)
+ + " pages are remaining.");
+ }
+
+ Set<MemorySegment> segmentsForOwner = allocatedSegments.get(owner);
+ if (segmentsForOwner == null) {
+ segmentsForOwner = new HashSet<MemorySegment>(numPages);
+ allocatedSegments.put(owner, segmentsForOwner);
+ }
+
+ if (isPreAllocated) {
+ for (int i = numPages; i > 0; i--) {
+ MemorySegment segment = memoryPool.requestSegmentFromPool(owner);
+ target.add(segment);
+ segmentsForOwner.add(segment);
+ }
+ }
+ else {
+ for (int i = numPages; i > 0; i--) {
+ MemorySegment segment = memoryPool.allocateNewSegment(owner);
+ target.add(segment);
+ segmentsForOwner.add(segment);
+ }
+ numNonAllocatedPages -= numPages;
+ }
+ }
+ // -------------------- END CRITICAL SECTION -------------------
+ }
+
+ /**
+ * Tries to release the memory for the specified segment. If the segment has already been released or
+ * is null, the request is simply ignored.
+ * <p>
+ * If the memory manager manages pre-allocated memory, the memory segment goes back to the memory pool.
+ * Otherwise, the segment is only freed and made eligible for reclamation by the GC.
+ *
+ * @param segment The segment to be released.
+ * @throws IllegalArgumentException Thrown, if the given segment is of an incompatible type.
+ */
+ public void release(MemorySegment segment) {
+ // check if segment is null or has already been freed
+ if (segment == null || segment.getOwner() == null) {
+ return;
+ }
+
+ final Object owner = segment.getOwner();
+
+ // -------------------- BEGIN CRITICAL SECTION -------------------
+ synchronized (lock)
+ {
+ // prevent double return to this memory manager
+ if (segment.isFreed()) {
+ return;
+ }
+ if (isShutDown) {
+ throw new IllegalStateException("Memory manager has been shut down.");
+ }
+
+ // remove the reference in the map for the owner
+ try {
+ Set<MemorySegment> segsForOwner = this.allocatedSegments.get(owner);
+
+ if (segsForOwner != null) {
+ segsForOwner.remove(segment);
+ if (segsForOwner.isEmpty()) {
+ this.allocatedSegments.remove(owner);
+ }
+ }
+
+ if (isPreAllocated) {
+ // release the memory in any case
+ memoryPool.returnSegmentToPool(segment);
+ }
+ else {
+ segment.free();
+ numNonAllocatedPages++;
+ }
+ }
+ catch (Throwable t) {
+ throw new RuntimeException("Error removing book-keeping reference to allocated memory segment.", t);
+ }
+ }
+ // -------------------- END CRITICAL SECTION -------------------
+ }
+
+ /**
+ * Tries to release many memory segments together.
+ * <p>
+ * If the memory manager manages pre-allocated memory, the memory segment goes back to the memory pool.
+ * Otherwise, the segment is only freed and made eligible for reclamation by the GC.
+ *
+ * @param segments The segments to be released.
+ * @throws NullPointerException Thrown, if the given collection is null.
+ * @throws IllegalArgumentException Thrown, id the segments are of an incompatible type.
+ */
+ public void release(Collection<MemorySegment> segments) {
+ if (segments == null) {
+ return;
+ }
+
+ // -------------------- BEGIN CRITICAL SECTION -------------------
+ synchronized (lock)
+ {
+ if (isShutDown) {
+ throw new IllegalStateException("Memory manager has been shut down.");
+ }
+
+ // since concurrent modifications to the collection
+ // can disturb the release, we need to try potentially multiple times
+ boolean successfullyReleased = false;
+ do {
+ final Iterator<MemorySegment> segmentsIterator = segments.iterator();
+
+ Object lastOwner = null;
+ Set<MemorySegment> segsForOwner = null;
+
+ try {
+ // go over all segments
+ while (segmentsIterator.hasNext()) {
+
+ final MemorySegment seg = segmentsIterator.next();
+ if (seg == null || seg.isFreed()) {
+ continue;
+ }
+
+ final Object owner = seg.getOwner();
+
+ try {
+ // get the list of segments by this owner only if it is a different owner than for
+ // the previous one (or it is the first segment)
+ if (lastOwner != owner) {
+ lastOwner = owner;
+ segsForOwner = this.allocatedSegments.get(owner);
+ }
+
+ // remove the segment from the list
+ if (segsForOwner != null) {
+ segsForOwner.remove(seg);
+ if (segsForOwner.isEmpty()) {
+ this.allocatedSegments.remove(owner);
+ }
+ }
+
+ if (isPreAllocated) {
+ memoryPool.returnSegmentToPool(seg);
+ }
+ else {
+ seg.free();
+ numNonAllocatedPages++;
+ }
+ }
+ catch (Throwable t) {
+ throw new RuntimeException(
+ "Error removing book-keeping reference to allocated memory segment.", t);
+ }
+ }
+
+ segments.clear();
+
+ // the only way to exit the loop
+ successfullyReleased = true;
+ }
+ catch (ConcurrentModificationException e) {
+ // this may happen in the case where an asynchronous
+ // call releases the memory. fall through the loop and try again
+ }
+ } while (!successfullyReleased);
+ }
+ // -------------------- END CRITICAL SECTION -------------------
+ }
+
+ /**
+ * Releases all memory segments for the given owner.
+ *
+ * @param owner The owner memory segments are to be released.
+ */
+ public void releaseAll(Object owner) {
+ if (owner == null) {
+ return;
+ }
+
+ // -------------------- BEGIN CRITICAL SECTION -------------------
+ synchronized (lock)
+ {
+ if (isShutDown) {
+ throw new IllegalStateException("Memory manager has been shut down.");
+ }
+
+ // get all segments
+ final Set<MemorySegment> segments = allocatedSegments.remove(owner);
+
+ // all segments may have been freed previously individually
+ if (segments == null || segments.isEmpty()) {
+ return;
+ }
+
+ // free each segment
+ if (isPreAllocated) {
+ for (MemorySegment seg : segments) {
+ memoryPool.returnSegmentToPool(seg);
+ }
+ }
+ else {
+ for (MemorySegment seg : segments) {
+ seg.free();
+ }
+ numNonAllocatedPages += segments.size();
+ }
+
+ segments.clear();
+ }
+ // -------------------- END CRITICAL SECTION -------------------
+ }
+
+ // ------------------------------------------------------------------------
+ // Properties, sizes and size conversions
+ // ------------------------------------------------------------------------
+
+ /**
+ * Gets the type of memory (heap / off-heap) managed by this memory manager.
+ *
+ * @return The type of memory managed by this memory manager.
+ */
+ public MemoryType getMemoryType() {
+ return memoryType;
+ }
+
+ /**
+ * Checks whether this memory manager pre-allocates the memory.
+ *
+ * @return True if the memory manager pre-allocates the memory, false if it allocates as needed.
+ */
+ public boolean isPreAllocated() {
+ return isPreAllocated;
+ }
+
+ /**
+ * Gets the size of the pages handled by the memory manager.
+ *
+ * @return The size of the pages handled by the memory manager.
+ */
+ public int getPageSize() {
+ return pageSize;
+ }
+
+ /**
+ * Returns the total size of memory handled by this memory manager.
+ *
+ * @return The total size of memory.
+ */
+ public long getMemorySize() {
+ return memorySize;
+ }
+
+ /**
+ * Gets the total number of memory pages managed by this memory manager.
+ *
+ * @return The total number of memory pages managed by this memory manager.
+ */
+ public int getTotalNumPages() {
+ return totalNumPages;
+ }
+
+ /**
+ * Computes to how many pages the given number of bytes corresponds. If the given number of bytes is not an
+ * exact multiple of a page size, the result is rounded down, such that a portion of the memory (smaller
+ * than the page size) is not included.
+ *
+ * @param fraction the fraction of the total memory per slot
+ * @return The number of pages to which
+ */
+ public int computeNumberOfPages(double fraction) {
+ if (fraction <= 0 || fraction > 1) {
+ throw new IllegalArgumentException("The fraction of memory to allocate must within (0, 1].");
+ }
+
+ return (int)(totalNumPages * fraction / numberOfSlots);
+ }
+
+ /**
+ * Computes the memory size of the fraction per slot.
+ *
+ * @param fraction The fraction of the memory of the task slot.
+ * @return The number of pages corresponding to the memory fraction.
+ */
+ public long computeMemorySize(double fraction) {
+ return pageSize * computeNumberOfPages(fraction);
+ }
+
+ /**
+ * Rounds the given value down to a multiple of the memory manager's page size.
+ *
+ * @return The given value, rounded down to a multiple of the page size.
+ */
+ public long roundDownToPageSizeMultiple(long numBytes) {
+ return numBytes & roundingMask;
+ }
+
+
+ // ------------------------------------------------------------------------
+ // Memory Pools
+ // ------------------------------------------------------------------------
+
+ static abstract class MemoryPool {
+
+ abstract int getNumberOfAvailableMemorySegments();
+
+ abstract MemorySegment allocateNewSegment(Object owner);
+
+ abstract MemorySegment requestSegmentFromPool(Object owner);
+
+ abstract void returnSegmentToPool(MemorySegment segment);
+
+ abstract void clear();
+ }
+
+ static final class HeapMemoryPool extends MemoryPool {
+
+ /** The collection of available memory segments */
+ private final ArrayDeque<byte[]> availableMemory;
+
+ private final int segmentSize;
+
+ public HeapMemoryPool(int numInitialSegments, int segmentSize) {
+ this.availableMemory = new ArrayDeque<byte[]>(numInitialSegments);
+ this.segmentSize = segmentSize;
+
+ for (int i = 0; i < numInitialSegments; i++) {
+ this.availableMemory.add(new byte[segmentSize]);
+ }
+ }
+
+ @Override
+ HeapMemorySegment allocateNewSegment(Object owner) {
+ return HeapMemorySegment.FACTORY.allocateUnpooledSegment(segmentSize, owner);
+ }
+
+ @Override
+ HeapMemorySegment requestSegmentFromPool(Object owner) {
+ byte[] buf = availableMemory.remove();
+ return HeapMemorySegment.FACTORY.wrapPooledHeapMemory(buf, owner);
+ }
+
+ @Override
+ void returnSegmentToPool(MemorySegment segment) {
+ if (segment.getClass() == HeapMemorySegment.class) {
+ HeapMemorySegment heapSegment = (HeapMemorySegment) segment;
+ availableMemory.add(heapSegment.getArray());
+ heapSegment.free();
+ }
+ else {
+ throw new IllegalArgumentException("Memory segment is not a " + HeapMemorySegment.class.getSimpleName());
+ }
+ }
+
+ @Override
+ protected int getNumberOfAvailableMemorySegments() {
+ return availableMemory.size();
+ }
+
+ @Override
+ void clear() {
+ availableMemory.clear();
+ }
+ }
+
+ static final class HybridOffHeapMemoryPool extends MemoryPool {
+
+ /** The collection of available memory segments */
+ private final ArrayDeque<ByteBuffer> availableMemory;
+
+ private final int segmentSize;
+
+ public HybridOffHeapMemoryPool(int numInitialSegments, int segmentSize) {
+ this.availableMemory = new ArrayDeque<ByteBuffer>(numInitialSegments);
+ this.segmentSize = segmentSize;
+
+ for (int i = 0; i < numInitialSegments; i++) {
+ this.availableMemory.add(ByteBuffer.allocateDirect(segmentSize));
+ }
+ }
+
+ @Override
+ HybridMemorySegment allocateNewSegment(Object owner) {
+ ByteBuffer memory = ByteBuffer.allocateDirect(segmentSize);
+ return HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(memory, owner);
+ }
+
+ @Override
+ HybridMemorySegment requestSegmentFromPool(Object owner) {
+ ByteBuffer buf = availableMemory.remove();
+ return HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(buf, owner);
+ }
+
+ @Override
+ void returnSegmentToPool(MemorySegment segment) {
+ if (segment.getClass() == HybridMemorySegment.class) {
+ HybridMemorySegment hybridSegment = (HybridMemorySegment) segment;
+ ByteBuffer buf = hybridSegment.getOffHeapBuffer();
+ availableMemory.add(buf);
+ hybridSegment.free();
+ }
+ else {
+ throw new IllegalArgumentException("Memory segment is not a " + HeapMemorySegment.class.getSimpleName());
+ }
+ }
+
+ @Override
+ protected int getNumberOfAvailableMemorySegments() {
+ return availableMemory.size();
+ }
+
+ @Override
+ void clear() {
+ availableMemory.clear();
+ }
+ }
+}