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 9ec71d21ca08c79d14d651e6b0f167ee93622891..bbaf71a502978c190af7a8157f9101ab3514d7b9 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 @@ -542,6 +547,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. */ 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 0000000000000000000000000000000000000000..0685d5928cd56abfb243854bacb7e41c5cb5a774 --- /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. + *

+ * 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. + *

+ * 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(); +} 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 0000000000000000000000000000000000000000..f68723b01419050977a6cb33126d2c27baed5e9f --- /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. + *

+ * 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. + *

+ * 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. + *

+ * 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. + *

+ * 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. + *

+ * 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. + *

+ * 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. + *

+ * 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(); +} 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 ce080d3f11f9b1c3819bf58d37122cacf26f9910..31d5563c8ddbbcb7dfac98eab5807279e58fcb83 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. + *

+ * 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. *

+ * This class fulfills conceptually a similar purpose as Java's {@link java.nio.ByteBuffer}. + * We add this specialized class for various reasons: + *

* - * Comments on the implementation: We make heavy use of operations that are supported by native + * Comments on the implementation: + * 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 reverseBytes methods in the boxing data types - * (for example {@link Integer#reverseBytes(int)}). On x86/amd64, these are translated by the - * jit compiler to bswap 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. + *

+ * 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 - * -XX:+UnlockDiagnosticVMOptions -XX:CompileCommand=print,*UnsafeMemorySegment.putLongBigEndian. + * -XX:+UnlockDiagnosticVMOptions -XX:CompileCommand=print,*MemorySegment.putLongBigEndian. * Note that this code realizes both the byte order swapping and the reinterpret cast access to * get a long from the byte array. - * + * *

  * [Verified Entry Point]
  *   0x00007fc403e19920: sub    $0x18,%rsp
@@ -66,286 +78,539 @@ import java.nio.ByteOrder;
  *                                                 ;   {poll_return}
  *   0x00007fc403e1994a: retq 
  * 
+ * + * Note on efficiency: + * 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 offset and * length 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 ByteBuffer 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 Integer.MAX_VALUE. Range checks that work after the principle - // position + 3 < end may fail because position + 3 becomes negative. - // A safe solution is to subtract the delta from the limit, for example - // position < end - 3. 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 DataOutput. - * - * @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)); + } } 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 0000000000000000000000000000000000000000..0e4e469dad1fa8871ca98cd3b7d2fd4d6c6a7018 --- /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. + *

+ * 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. + *

+ * 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. + *

+ * 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. + *

+ * 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 direct ByteBuffer. + *

+ * 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. + *

+ * 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 direct ByteBuffer. + *

+ * 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); + } +} 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 0000000000000000000000000000000000000000..5b2b5d343ba7a9e3b1083813d8b5ea7c2275baec --- /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 +} 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 c1f626f2ba8fe9455738052df857182942c1b8ea..20b37c9c9a60c03107d3634ba6a1e29a8578dcef 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() {} } 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 782c402f1addfe31a78737aeeddfaa4192e416d5..a8ace92f9f77d8fcc25e3eed537a738ce266e8f1 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 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 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 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 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 { 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 0000000000000000000000000000000000000000..724a366f79998a7ef819e6896e9dab3cbe1bb3d3 --- /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) {} + } + } + } + } +} 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 0000000000000000000000000000000000000000..4c92234bae9be35aa538f800d02ffbcd8a11a37f --- /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); + } + } + } +} 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 0000000000000000000000000000000000000000..d7a5b03b8922cd08aa665f4aa8b83f4aa56720d6 --- /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()); + } + } +} 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 0000000000000000000000000000000000000000..b09697ee4141eb802a2dda2f62dcf2a2249c541c --- /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()); + } + } +} 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 0000000000000000000000000000000000000000..55d333e23ec95286d3aaa69491132f9bdced0124 --- /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()); + } + } +} 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 0000000000000000000000000000000000000000..f50322cd11072528f723c73faf1952f7da5e0488 --- /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) {} + }; +} 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 0000000000000000000000000000000000000000..10c362284ece085411ff292552bf63e08ec5da26 --- /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 executionModes(){ + return Arrays.asList( + new Object[] { 32*1024 }, + new Object[] { 4*1024 }, + new Object[] { 512*1024 } ); + } +} \ No newline at end of file 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 0000000000000000000000000000000000000000..7b8d9961ef448dea45d85d1754a0c030b148d4ec --- /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); + } + } +} 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 0000000000000000000000000000000000000000..ddbdf72815e08b48337c5016c6375fc901720ad6 --- /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) {} + } + } + } + } +} 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 0000000000000000000000000000000000000000..df3f8beb942efb7492397444473263d4ecb5626c --- /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 memorySource; + + private final List fullSegments; + + + private byte[] utfBuffer; // the reusable array for UTF encodings + + + public CoreMemorySegmentOutView(List emptySegments, + List 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(); + } + } +} 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 0000000000000000000000000000000000000000..ed3c1f5469864aff81fa34c81f641a3564f8dffc --- /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 memory = new ArrayList<>(); + memory.add(coreHeap); + ArrayList 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 memory = new ArrayList<>(); + memory.add(coreHybridOnHeap); + ArrayList 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 memory = new ArrayList<>(); + memory.add(coreHybridOffHeap); + ArrayList 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 memory = new ArrayList<>(); + memory.add(pureHeap); + ArrayList 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 memory = new ArrayList<>(); + memory.add(pureHybridOnHeap); + ArrayList 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 memory = new ArrayList<>(); + memory.add(pureHybridOffHeap); + ArrayList 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); + } + } +} 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 0000000000000000000000000000000000000000..454c821dea2663597314307d478af2af1e6338f0 --- /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(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/UnsafeMemorySegment.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegment.java similarity index 54% rename from flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/UnsafeMemorySegment.java rename to flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegment.java index 21d3e27c0e8b5dfad958077e075411ba7d010ec0..e247eed85730f63d6c560037e961aca40c5fe009 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/UnsafeMemorySegment.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegment.java @@ -16,8 +16,9 @@ * limitations under the License. */ +package org.apache.flink.core.memory.benchmarks; -package org.apache.flink.runtime.memorymanager; +import org.apache.flink.core.memory.MemoryUtils; import java.io.DataInput; import java.io.DataOutput; @@ -25,207 +26,228 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import org.apache.flink.core.memory.MemoryUtils; +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; -/** - * 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. + * Creates a new memory segment that represents the data in the given byte array. + * + * @param memory The byte array that holds the data. */ - public UnsafeMemorySegment(byte[] memory) { + public PureHeapMemorySegment(byte[] memory) { this.memory = memory; + this.size = memory.length; } // ------------------------------------------------------------------------- - // MemorySegment Accessors + // Direct Memory Segment Specifics // ------------------------------------------------------------------------- - + /** - * 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. + * 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; } - /** - * 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; + public final void free() { + this.wrapper = null; + this.memory = null; } - /** - * 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; + public final int size() { + return this.size; } - // ------------------------------------------------------------------------- - // Helper methods - // ------------------------------------------------------------------------- - - - /** - * Wraps the chunk of the underlying memory located between offset and - * length 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 ByteBuffer 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) { + 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.position(offset); this.wrapper.limit(offset + length); + this.wrapper.position(offset); } - + 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 Integer.MAX_VALUE. Range checks that work after the principle - // position + 3 < end may fail because position + 3 becomes negative. - // A safe solution is to subtract the delta from the limit, for example - // position < end - 3. Since all indices are always positive, and the integer domain - // has one more negative value than positive values, this can never cause an underflow. - // ------------------------------------------------------------------------------------------------------ - + + // ------------------------------------------------------------------------ + // 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 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); } - + + @SuppressWarnings("restriction") public final char getChar(int index) { - return (char) ( ((this.memory[index ] & 0xff) << 8) | - (this.memory[index + 1] & 0xff) ); + 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) { - this.memory[index ] = (byte) (value >> 8); - this.memory[index + 1] = (byte) 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) { - return (short) ( - ((this.memory[index ] & 0xff) << 8) | - ((this.memory[index + 1] & 0xff)) ); + 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) { - this.memory[index ] = (byte) (value >> 8); - this.memory[index + 1] = (byte) 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 (CHECKED) { - if (index >= 0 && index <= this.memory.length - 4) { - return UNSAFE.getInt(this.memory, BASE_OFFSET + index); - } else { - throw new IndexOutOfBoundsException(); - } - } else { + 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); @@ -241,20 +263,16 @@ public class UnsafeMemorySegment { 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 { + 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); @@ -273,17 +291,13 @@ public class UnsafeMemorySegment { @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 { + 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); @@ -302,14 +316,10 @@ public class UnsafeMemorySegment { @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 { + if (index >= 0 && index <= this.memory.length - 8) { UNSAFE.putLong(this.memory, BASE_OFFSET + index, value); + } else { + throw new IndexOutOfBoundsException(); } } @@ -328,7 +338,7 @@ public class UnsafeMemorySegment { putLong(index, value); } } - + public final float getFloat(int index) { return Float.intBitsToFloat(getInt(index)); } @@ -340,7 +350,7 @@ public class UnsafeMemorySegment { public final float getFloatBigEndian(int index) { return Float.intBitsToFloat(getIntBigEndian(index)); } - + public final void putFloat(int index, float value) { putInt(index, Float.floatToRawIntBits(value)); } @@ -360,32 +370,97 @@ public class UnsafeMemorySegment { 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 + // 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); } 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 0000000000000000000000000000000000000000..1e3b89e9c1ef3b7e173b1227a94b7ccb51ee2955 --- /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 memorySource; + + private final List fullSegments; + + + private byte[] utfBuffer; // the reusable array for UTF encodings + + + public PureHeapMemorySegmentOutView(List emptySegments, + List 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(); + } + } +} 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 0000000000000000000000000000000000000000..57817b9da6df32d0b6e1d54524323e52655e654c --- /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); + } + } +} 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 0000000000000000000000000000000000000000..cda48e14fbca7a3abb8e8dd31b2634868d401251 --- /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 memorySource; + + private final List fullSegments; + + + private byte[] utfBuffer; // the reusable array for UTF encodings + + + public PureHybridMemorySegmentOutView(List emptySegments, + List 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(); + } + } +} 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 0000000000000000000000000000000000000000..1280242188d3cc6c8710a897a6c3b68829f18005 --- /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); + } + } +} 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 0000000000000000000000000000000000000000..d898dee7b7bc546a2094798faa995aeb0c4e0c30 --- /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 memorySource; + + private final List fullSegments; + + + private byte[] utfBuffer; // the reusable array for UTF encodings + + + public PureOffHeapMemorySegmentOutView(List emptySegments, + List 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(); + } + } +} 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 0000000000000000000000000000000000000000..2163eb56d15e31d9eb1298b00843471b4fb27439 --- /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 memory = new ArrayList<>(); + memory.add(HeapMemorySegment.FACTORY.wrapPooledHeapMemory(largeSegment, null)); + ArrayList 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 memory = new ArrayList<>(); + memory.add(HybridMemorySegment.FACTORY.wrapPooledHeapMemory(largeSegment, null)); + ArrayList 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 memory = new ArrayList<>(); + memory.add(HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(largeOffHeap, null)); + ArrayList 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 memory = new ArrayList<>(); + memory.add(new PureHybridMemorySegment(largeSegment)); + ArrayList 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 memory = new ArrayList<>(); + memory.add(new PureHybridMemorySegment(largeOffHeap)); + ArrayList 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 memory = new ArrayList<>(); + memory.add(new PureHeapMemorySegment(largeSegment)); + ArrayList 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; + } +} 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 b39fb2ee44e7f8d0dc3f49e2f9f3406d31d5988f..d6c15d9b85f9bf37e31c6d43ad93eaee9a84c8e0 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 > void assertNormalizableKey(NormalizableKey key1, NormalizableKey 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"); } 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 c742ce5095ff354de7098d7eba05818866e4e668..87d89eb35bde806527e2b6fe599211451965e898 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; 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 736c245e4c42aea33bfda875851b4bbd0032bb89..2ca7f78b80cb4cf0c07f8ccac6ab2cae79a71333 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; /** 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 b6c500f9c58d171259964d02555343ec47eed015..8415d092268bf304bcf1158c7770642e5d5257b3 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 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 718b8af2fe9ecf06ae76eae369073a2b79886d33..23dccb083403c17c19e4bddd5e192262e7f33b1b 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; 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 6ed9f4a7f8c01645752f9ce4287969fc6b372d2a..427fe844126c630339df51315a6941eb37000091 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; 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 7d8d48561510dccece7761b46ab459df628978f0..355b2ebfa319335480e4208a2405326e3d180d81 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; /** 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 35ec3e3f52dd0308e7e22ea4bd2a8c9366209989..a107e79eaefaa8ae864849670d8dd1fbbc88a327 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; /** 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 5f9c2cffed64420cbcb115a572d63f1d514658ca..397c40caa9aced1881cec667bff2f8c02e829f7f 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; /** 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 b9190345eb5b199a8e66fd05124e3af94a96bea7..de1ed876a0003d6789299724e0404e87b04b5c9f 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 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 089e10a78228ecb19ce5208463bf20c583cc057a..64b2ebce6f90a782e2a70d9a23bc97b4d5db5dd1 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 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 9eb4a9afd59766de6f1cd056572b40b2f9ded9f1..6e4b5648f0b02f46846b497fbb6fc50faf3d2cad 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); 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 ec9f4fdccff38d985994c0f56adc666fc8ef9a31..cdd8731c3a74cb7712abea213d2872dd45bfdccd 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 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; } 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 b23b83b0987085cd84f01e652a4b4ccf8041de6e..a34f8cfe42673cd4f31dfb03ab94f8b0fe624f0e 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; 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 f3e4892fe32123667330ce2ce8dc2fe5473700c4..b10e5a8d9ed81546b631d5f4841c2a8f3c23293e 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(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; 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 3b7d921024dccdd5f5a346586260872ea816e9c1..ee015c20efd4c699d58e3b66dc6d43d780ef15b8 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); 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 5b91668e8107ff776a4271eb770cac08db3d270a..2c0c74473fff5e2cb1874252e7318ea39cb322da 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)); } } } 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 ca1ade90ff719c43df09c3bfd3534594ac4c03e7..77768940fb84f2c20346fb3fa59ebb913923d979 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 { 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/memory/AbstractPagedInputView.java similarity index 97% rename from flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedInputView.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedInputView.java index 23081c90f4931e4c652afb659309e876047f9dc2..788eebba45070e22a14d383170e43534e765f07c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedInputView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedInputView.java @@ -17,7 +17,7 @@ */ -package org.apache.flink.runtime.memorymanager; +package org.apache.flink.runtime.memory; import java.io.EOFException; import java.io.IOException; @@ -211,7 +211,8 @@ public abstract class AbstractPagedInputView implements DataInputView { if (remaining == 0) { try { advance(); - }catch(EOFException eof){ + } + catch (EOFException eof) { return -1; } remaining = this.limitInSegment - this.positionInSegment; @@ -227,7 +228,8 @@ public abstract class AbstractPagedInputView implements DataInputView { if (len > bytesRead) { try { advance(); - }catch(EOFException eof){ + } + catch (EOFException eof) { this.positionInSegment += toRead; return bytesRead; } @@ -280,7 +282,7 @@ public abstract class AbstractPagedInputView implements DataInputView { @Override public short readShort() throws IOException { if (this.positionInSegment < this.limitInSegment - 1) { - final short v = this.currentSegment.getShort(this.positionInSegment); + final short v = this.currentSegment.getShortBigEndian(this.positionInSegment); this.positionInSegment += 2; return v; } @@ -296,7 +298,7 @@ public abstract class AbstractPagedInputView implements DataInputView { @Override public int readUnsignedShort() throws IOException { if (this.positionInSegment < this.limitInSegment - 1) { - final int v = this.currentSegment.getShort(this.positionInSegment) & 0xffff; + final int v = this.currentSegment.getShortBigEndian(this.positionInSegment) & 0xffff; this.positionInSegment += 2; return v; } @@ -312,7 +314,7 @@ public abstract class AbstractPagedInputView implements DataInputView { @Override public char readChar() throws IOException { if (this.positionInSegment < this.limitInSegment - 1) { - final char v = this.currentSegment.getChar(this.positionInSegment); + final char v = this.currentSegment.getCharBigEndian(this.positionInSegment); this.positionInSegment += 2; return v; } 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/memory/AbstractPagedOutputView.java similarity index 96% rename from flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedOutputView.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedOutputView.java index ea73e62c3f1889aa4dfebd56871cd65f563ab10e..482d82e0210121ad29239f7d4ce7c4d877cc1752 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedOutputView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedOutputView.java @@ -17,7 +17,7 @@ */ -package org.apache.flink.runtime.memorymanager; +package org.apache.flink.runtime.memory; import java.io.IOException; import java.io.UTFDataFormatException; @@ -228,7 +228,7 @@ public abstract class AbstractPagedOutputView implements DataOutputView { @Override public void writeShort(int v) throws IOException { if (this.positionInSegment < this.segmentSize - 1) { - this.currentSegment.putShort(this.positionInSegment, (short) v); + this.currentSegment.putShortBigEndian(this.positionInSegment, (short) v); this.positionInSegment += 2; } else if (this.positionInSegment == this.segmentSize) { @@ -244,7 +244,7 @@ public abstract class AbstractPagedOutputView implements DataOutputView { @Override public void writeChar(int v) throws IOException { if (this.positionInSegment < this.segmentSize - 1) { - this.currentSegment.putChar(this.positionInSegment, (char) v); + this.currentSegment.putCharBigEndian(this.positionInSegment, (char) v); this.positionInSegment += 2; } else if (this.positionInSegment == this.segmentSize) { @@ -349,9 +349,9 @@ public abstract class AbstractPagedOutputView implements DataOutputView { final byte[] bytearr = this.utfBuffer; bytearr[count++] = (byte) ((utflen >>> 8) & 0xFF); - bytearr[count++] = (byte) ((utflen >>> 0) & 0xFF); + bytearr[count++] = (byte) ( utflen & 0xFF); - int i = 0; + int i; for (i = 0; i < strlen; i++) { c = str.charAt(i); if (!((c >= 0x0001) && (c <= 0x007F))) { @@ -368,10 +368,10 @@ public abstract class AbstractPagedOutputView implements DataOutputView { } 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)); + bytearr[count++] = (byte) (0x80 | ( c & 0x3F)); } else { bytearr[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F)); - bytearr[count++] = (byte) (0x80 | ((c >> 0) & 0x3F)); + bytearr[count++] = (byte) (0x80 | ( c & 0x3F)); } } @@ -390,7 +390,6 @@ public abstract class AbstractPagedOutputView implements DataOutputView { advance(); numBytes -= remaining; } - return; } @Override @@ -411,6 +410,5 @@ public abstract class AbstractPagedOutputView implements DataOutputView { advance(); } - return; } } 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/memory/ListMemorySegmentSource.java similarity index 96% rename from flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/ListMemorySegmentSource.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/memory/ListMemorySegmentSource.java index cacb6b04d88f703ff518b72c132297a40f2ffee8..ebeb47df43641ecaf026102711eb236d92c43822 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/ListMemorySegmentSource.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/ListMemorySegmentSource.java @@ -17,7 +17,7 @@ */ -package org.apache.flink.runtime.memorymanager; +package org.apache.flink.runtime.memory; import java.util.List; 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/memory/MemoryAllocationException.java similarity index 96% rename from flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryAllocationException.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryAllocationException.java index d4d8b3b686ec72c6f384eb01131a369023bf27a7..112d6f7474bf966c31f8761ff558b8f06b51351e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryAllocationException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryAllocationException.java @@ -17,7 +17,7 @@ */ -package org.apache.flink.runtime.memorymanager; +package org.apache.flink.runtime.memory; /** * An exception to be thrown when a memory allocation operation is not successful. 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 0000000000000000000000000000000000000000..094d065abe83777370e864e8e6877879e78f5a3a --- /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. + *

+ * 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. + *

+ * 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> 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>(); + 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 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 allocatePages(Object owner, int numPages) throws MemoryAllocationException { + final ArrayList segs = new ArrayList(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 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) 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 segmentsForOwner = allocatedSegments.get(owner); + if (segmentsForOwner == null) { + segmentsForOwner = new HashSet(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. + *

+ * 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 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. + *

+ * 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 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 segmentsIterator = segments.iterator(); + + Object lastOwner = null; + Set 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 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 availableMemory; + + private final int segmentSize; + + public HeapMemoryPool(int numInitialSegments, int segmentSize) { + this.availableMemory = new ArrayDeque(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 availableMemory; + + private final int segmentSize; + + public HybridOffHeapMemoryPool(int numInitialSegments, int segmentSize) { + this.availableMemory = new ArrayDeque(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(); + } + } +} 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 85c91c006587c4735c1b4a0868ce24226247bd63..0000000000000000000000000000000000000000 --- 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 Integer.MAX_VALUE. Range checks that work after the principle - // position + 3 < end may fail because position + 3 becomes negative. - // A safe solution is to subtract the delta from the limit, for example - // position < end - 3. 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; - } -} 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 b041ac959b992f75eec2fbb33169356b91ccf2d6..0000000000000000000000000000000000000000 --- 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 freeSegments; // the free memory segments - - private final HashMap> 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(this.totalNumPages); - this.allocatedSegments = new HashMap>(); - - 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 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 allocatePages(AbstractInvokable owner, int numPages) throws MemoryAllocationException { - final ArrayList segs = new ArrayList(numPages); - allocatePages(owner, segs, numPages); - return segs; - } - - @Override - public void allocatePages(AbstractInvokable owner, List 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) 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 segmentsForOwner = this.allocatedSegments.get(owner); - if (segmentsForOwner == null) { - segmentsForOwner = new HashSet(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 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 void release(Collection 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 segmentsIterator = segments.iterator(); - - AbstractInvokable lastOwner = null; - Set 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 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; - } - } -} 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 631f0b8b08abc1df699c55c2ee2e96967b1f838b..0000000000000000000000000000000000000000 --- 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. - *

- * 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 allocatePages(AbstractInvokable owner, int numPages) throws MemoryAllocationException; - - void allocatePages(AbstractInvokable owner, List target, int numPages) throws MemoryAllocationException; - - /** - * Tries to release the memory for the specified segment. If the segment has already been released or - * is null, the request is simply ignored. If the segment is not from the expected - * MemorySegment implementation type, an IllegalArgumentException 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 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. - */ - void release(Collection 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(); -} 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 7219503c40f86fe0e7ea1e60ad7b9b2947d84cfd..0000000000000000000000000000000000000000 --- 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 offset and - * length 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 ByteBuffer 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 Integer.MAX_VALUE. Range checks that work after the principle - // position + 3 < end may fail because position + 3 becomes negative. - // A safe solution is to subtract the delta from the limit, for example - // position < end - 3. 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)); - } -} 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 89b7709c53b81fe9a3231f42f55086837c44a64c..38c74e011b87710c11532cf70d0270be439a46f3 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; 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 cc3934124c2831bbe74a90bdc7d368e741a62771..011c98004fc17a53d550085060f5b769084f8c43 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; 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 998064a1c9f702e59efcd04c5396b85a7d5f46e7..30786aa0014b28ae331d38566b01765d01160de3 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; 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 2bf778e61fdbfa384c4ff757394afb657a667276..028ed95c57e13ed9fb9c4f749a95c587934a96ec 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; 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 95e98ce664f7ad748d2b30f3cac74e7c86d6e39e..811f00c0841760397f034570fd11af0c449edbfc 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; 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 b8932af4a638e1f7bb75394db283e9052c6dd9fb..3cccab82dc978fdf3cc362208415de1e5a90eaa4 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; 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 5c2ed679e0657405c2d365851b4e199cf233e98a..baeda3a2e0aa5975cd69af94b0acd656d3604e11 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; 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 f3ea7004d1cbc95e8242140cfc853bbe3051497d..19557bc5f26c0df5516370c7ed9f3b9e37d5c694 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; 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 3cefbba67a0005403a533ad95a1671db3d9e98b6..6d35f929423f629c6f8458bfa02bf2884dac54eb 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; 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 c455dc4e84ce4686785c915cd23e5e8437066e6d..c93637ec84d7da8477657d4471524d11965beaf8 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; 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 63f2fd519209b13639ad5e17eb7b3f88c019c157..59a0f59f8b68b85bfdb9bb667dac130739ba8df9 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; 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 ff397c2ee573877f9be13bdc0eb64cc6e87c96e7..09f4288e35eb99bf386a544e33868804f45bf247 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; 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 41161454dc4ae841c38b1992440bfc280508b6df..75c1eed164e3540b9a58a0fba74930098818472c 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; 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 ff6548e059288edf3d3fc7b76983a4a167f6f13e..5a0c6cc7515764c65906f2576753f989c7d32e2d 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; 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 08b619103b6a6805b235e792afce1a3fedc73289..3b12c68e2f40c678a0bfca6968cbc8feeb771f18 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; 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 b899acdd3662f70fb9021358d8f582f8ad57f05a..7baaee76b90bbb45c3d23854041cacd8c0e184f5 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; 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 ffb66fc3b85e34cf0a13a369e406554909607e04..7554bc346c9bbfd537fc4ca05dcf84591ef7db55 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} 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 5000dab591008cdcb6626483edf521d53b4afdf2..dbdb5b251e03bafbe5e061a82bb2931878960335 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; 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 af1626a376821b15bab37dc5e2c600cc7d1beb5b..b51c3b109a02d7ddaa541613475e3feff862b175 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; 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 83952c93b41f4a5c3e43032f063d75dfaa29c236..26dba7b7c97f928d1f6cca473e3e502136f0a9ed 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; 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 029be5ffd48387ae941e8397dae42b8e4070ab76..92b0fffc5ff1e0baa9ac81179de6543017540d98 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; 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 b4aaa9536948914a3e8a35492f331295505b5a6a..65dfd899c1abc01be9c9388ab66fe3a2a0211473 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; 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 714a1f57a8094a3d02133bf16cc101ee5e972ec1..5635865da4adbf61989117c000582aea7f9d1883 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 extends ReusingBuildFirstHashMatchIterator { 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 b7c3e29662bed2ee9c0886442ad2985a4b823db8..156f259c3c7f7c0bd2c4b530686d89f92d65c963 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; 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 4b4cdf57d800d7649160f7b3dba2a6c08e599846..a0791fec481fd45a7d4e441a0231b228c12867b1 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 extends ReusingBuildSecondHashMatchIterator { 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 730d19a8d2df9f8015b066def2cc6540622e9352..859757f96eb8552363569331c5a01facac555633 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; /** 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 abce462ab4a08c474ad9a5662e593e817a3dbde3..124ba558457bee1b5d5d637cf0af490d3f7c320c 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; 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 9d581ce661453bcee9a109824f5fa6cf53546819..408d6e4658cf127dafaff60e9bd7413604359ebe 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; /** 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 baa0fb2b4357f95b54de0305042a9f6c783c824c..ff580922626addb9294e07431813e76f9662ca5a 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; 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 e5a4501c5c8346fba2ebd4aef0b15e758923f793..c59b52ab089a8630dfc63fac8f3d2c398998416c 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; /** 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 5467ae91ac7ffd860ad350db7beec817eabf596b..12d5400630acc7f091a692f9faa74ea4e5b50993 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; 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 e9ccf521260bd9c2ddde475eb5c5e030c12c3a6b..db56227369bd3bb970c8e8b6d41ab9a99c7dfe7c 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; 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 68e0d47a6cade298538d40d55629072d9e4edb55..a6bd4d0d67a6ab2dab02547222d5cfc9b58394fe 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; 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 f34639ff3e9137246056930eac6a643a1a66b4a8..d109cf8c393e794d7edbfdb27ce94b044f831407 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; 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 8da9413146d33469c5a056811b1bda679aa6ed6c..f662a7e42b984011ea6c80237fbda3d738f3b4b1 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; 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 cd982c4084e93275d1f4d62fd22e12991ab7cb5a..e9325927259a3db8bfbb6abc1d8d021f84a4f5c1 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 implements InMemorySorter { // 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 implements InMemorySorter { 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 implements InMemorySorter { 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 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 ce630211409bc5f34ed701b5c5c29549b1d92a50..e4a99fb9967a153c081a9700b05c8f1ebf7197a6 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; 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 644084cdc45207def747a8d9dae526463ec75940..d2083986e70f6f7efa2df0876152f16512287b60 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; 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 33510d304ee2d365718f4e68bcda186ad9c60449..db47f168526e6a398b8ef3bc2469b1a4218d32e6 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; 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 fe877887792b6b6eb488a39e1c3c7ec9f5195483..c05e518db2dd26f9977a7daa97ea1b11a76db7dd 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 implements InMemorySorter { // 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 implements InMemorySorter { 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 implements InMemorySorter { } } - 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 implements InMemorySorter { 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 implements InMemorySorter { 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 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 3a1a17a9205d4dfd1d974d47099b4546a6447ff7..6b8fcf9d89b7ff507c859c60820f33a89e807072 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; 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 ce7bab4dfdabbee318b5c745a7205ba447ebb4fd..8382b868f61ca44ca2c5dd995fb16800f0bde1bd 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; 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 388b7b9743c79467bd14bb19241277d4f4aed364..fd1062d5d2641a87f55668476456b75ba37c3e59 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; 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 615279752a16c594b765d03e52f92007632795d5..ac3668b230c29ce5f3710cf8423533657e3126c4 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. 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 1f1d625df74576d2db56056785ba81886bfc65e7..197eaf8cbdf2a9acebd776028151e901ebab7047 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; /** 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 07cea33f51b9543d51a40258ffc068b97f27c509..567f05157170718c624a4d178a51bb4dbc1ceaf9 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; 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 a19a57a156453e855117882df912c98134c404b3..0d1dd13e3b9625f5ca9b1c0c645366b345f92d17 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; 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 f24302763d3690c777e2165a00f414cde96c2b59..b8519b09f76e119a6168fa1571d841be10fcc90f 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; } /** 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 5bcda60ad352eb24a27f949cb604a924fe553f74..065211c934e2b9635659b0ecc92a533e6e36d31b 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)) 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 529d3d13ad647a38856ec4d58635f6e6f7243afe..f11b933f2f26188ab45dd58a62e6f40711b5eba0 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) } /** 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 f2a5e2d3ce743ea4152c2380e6e48c136fa37bc9..a44916ab5ca7b49e574c3f5890a838d4ce191a0d 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(); } 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 dcc1e5f95b28855f32074d9414a1cde8e9bd06fe..5deb50eb431521ba0393347097d67dde52fdb8d9 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(); } 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 f9b8b38160602bc3ee7fa2734f4bd74c4ab5a473..1c2b3dee7c80ae693f748fe406ba0f5ca3325393 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 memory = new ArrayList(); 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 memory = new ArrayList(); memMan.allocatePages(new DummyInvokable(), memory, 4); 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 c071bef3d413e6b7dc667adbc77ea480bb94edc1..4c6a2b317e96e88981120bc7bf18b6316b8b9449 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 memory = new ArrayList(); 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(); 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 6a9a20a837e5715a4e026a98ef64a5e5e5f98187..0b1e0c3e6358e6a4594b9aacb2771aedcc08f49e 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 memory = new ArrayList(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 memory = new ArrayList(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 memory = new ArrayList(NUM_MEMORY_SEGMENTS); 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 49e93c6cc699096f5d610c9a039a7af1d6b743dd..a471e663ea3037cb1a8497d38606c07d5e61f0c4 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 writer = new AsynchronousBlockWriterWithCallback(channelId, @@ -371,7 +373,7 @@ public class AsynchronousFileIOChannelTest { finally { try { writer.closeAndDelete(); - } catch (Throwable t) {} + } catch (Throwable ignored) {} } } catch (Exception e) { 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 294a6e6dd9b51b8dbc50a9767ce7fca628dc5a85..c1bd4655bd85aaa9757cb8d72f2eac9f25a0f840 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) { 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 b0c702a14b484da1286b807a3d228c8c53e2c55a..24d2864dbedf132b4d201d791ef741c647d4f86a 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) { 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 435588fb13cd85e3837a08d3eb8022f13df6294a..4656d5662a91a0d70e4c4263a51841e4cc7d2da1 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 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 memSegs = new ArrayList(); 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(); 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 52908d350d37402bd415886c0a2fe1c3476629ae..6c25117367cddadedbfb52904ae41ece293f0d38 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; 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 3bdc9bd6a9dbb09c2a48d5de79c29a11c2c4f9ed..fd02623af5473810ccfe058337dfd3905ff934c9 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(); } 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 d657ebfaa46584393b00c64b2a6759a1040b5fcc..420199cce68755b0fe764ed79403beab94ddb8a1 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(nettyConf), + NUM_BUFFERS, BUFFER_SIZE, MemoryType.HEAP, + IOManager.IOMode.SYNC, new Some(nettyConf), new Tuple2(0, 0)); NetworkEnvironment env = new NetworkEnvironment( 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 e7d05240d8ab22e0a880df08b623bf4b024793c0..8455402eb35aaab848a53df2cd206af160b7107e 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() { 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 cd6d58090e9b355d37be869d02fa97db72dffa2e..819a94ff735c0be0b31e31e6c9d56bf2fdf86236 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 serializedRecords = new ArrayDeque(); 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 50d36392c1c09fc7304a3b158f9cbcb73c1083af..b7bcb3e6b16d771bf973a08ea09a4c518040f9ab 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 serializer = new SpanningRecordSerializer(); - 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 serializer = new SpanningRecordSerializer(); - 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 serializer = new SpanningRecordSerializer(); - final Buffer buffer = new Buffer(new MemorySegment(new byte[segmentSize]), mock(BufferRecycler.class)); + final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(segmentSize), mock(BufferRecycler.class)); // ------------------------------------------------------------------------------------------------------------- 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 9e1058268cea9dc47a9bf16f4f66408628f4ec60..f8cd28f2e327d190059771168e6917f58aa215ad 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); 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 28862e8bbdb3ac49ea6b377b4f6d9b579382e3e5..0ac84dccb72f4c7f3b923d48e5f3a27ab64b0264 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 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 734dcfb45c787f5b820a7126afa08dac9eac7ba6..fd11d029d96617574437bb7682bb6ba749e62d36 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); 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 e8e9ec84c87ab4148942bde2c319c1fa4f5ba520..93731e1ecf424f4e2dbbe2f7fe7eed19c0192eca 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()); 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 6b22cd930b89e610fa48664d3d2bc3d0edd29f2b..fd5c7a5dd8e0fd8ac4fba65f73dc84422c7e73f3 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); 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 60241e3b7856cdd4c7899bcf28ee3e81bbc8803f..f514cbd47405a44ecff477dd1d02094b52b988a9 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) { 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 65780b4ed8d9ea1af2f1774ec1f6a022b933bbf5..cfbe99e833a07c233599da67f3404db96475a7cd 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 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); 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 cd56318c46b2207891bfd025c4604e5c285f15c7..ea40a55b67e5928cc9d63dfc67bf63cb0c9e92cb 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); 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 82cc730300aed7da20a97b9143561c886c8386a6..f4c37f4f899072edb48e1b079eadb9c7af39c6cb 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); 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 d9e3562920116d86d8bb374601e093f6e0b6e64d..d628596e6ea6b5d83a4707393b8b6670e028e1bc 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 serializer = new SpanningRecordSerializer(); final RecordDeserializer deserializer = new AdaptiveSpanningRecordDeserializer(); - 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 originalRecords = new ArrayList(); List deserializedRecords = new ArrayList(); @@ -149,7 +149,7 @@ public class LargeRecordsTest { final RecordSerializer serializer = new SpanningRecordSerializer(); final RecordDeserializer deserializer = new SpillingAdaptiveSpanningRecordDeserializer(); - 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 originalRecords = new ArrayList(); List deserializedRecords = new ArrayList(); 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 cdba545977a31ae292190102fe79bdef49d4b853..4b3b465a396693a736823283a3c1c54e281dda1f 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() { 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 fc88207a70df39327beb2556cf066c92183a0d77..15251e974d4bb7e47bf7a3d48c6c4c7cc1badcd2 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); } 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 c0f32ca1dd9c8bae64b6d2fb7c531faa3a4d5a0b..a20a180b6a7e7ac6bc972ea93ed02f659e32c136 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); } 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/MemorySegmentSimpleTest.java similarity index 97% rename from flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSimpleTest.java index 990dffa6bc1df09dd80994235d5e358c849a0e48..fad1b0e8f26bfb0cfceca0eb89a79f50567d57e2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSimpleTest.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.runtime.memory; import static org.junit.Assert.assertArrayEquals; @@ -27,15 +26,17 @@ 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.junit.Assert; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.memorymanager.DefaultMemoryManager; + +import org.junit.Assert; import org.junit.After; import org.junit.Before; import org.junit.Test; -public class MemorySegmentTest { +public class MemorySegmentSimpleTest { public static final long RANDOM_SEED = 643196033469871L; @@ -43,7 +44,7 @@ public class MemorySegmentTest { public static final int PAGE_SIZE = 1024 * 512; - private DefaultMemoryManager manager; + private MemoryManager manager; private MemorySegment segment; @@ -52,7 +53,7 @@ public class MemorySegmentTest { @Before public void setUp() throws Exception{ try { - this.manager = new DefaultMemoryManager(MANAGED_MEMORY_SIZE, 1, PAGE_SIZE, true); + 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) { @@ -550,7 +551,7 @@ public class MemorySegmentTest { @Test public void testByteBufferWrapping() { try { - MemorySegment seg = new MemorySegment(new byte[1024]); + MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(1024); ByteBuffer buf1 = seg.wrap(13, 47); assertEquals(13, buf1.position()); 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 0843cc26cd393f2232485a572565e4d2a454d9fc..0000000000000000000000000000000000000000 --- 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; - } -} 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 30e417b87c71ed0a77cee7fe188d040fd33b4399..0a02f306fb93b499ce016fd60f43faba21e8873d 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 implements PactTaskContext { 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()); } 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 e3b697e6d9e2d423f1e12ff7b2d2ab71bd1fdcc6..abcbf732b9b2698353403e4a44a7dd996b4b5071 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 getMemory(int numSegments, int segmentSize) { ArrayList list = new ArrayList(numSegments); for (int i = 0; i < numSegments; i++) { - list.add(new MemorySegment(new byte[segmentSize])); + list.add(MemorySegmentFactory.allocateUnpooledSegment(segmentSize)); } return list; } 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 52f6ffc8c88ce5c3c658fab06e5474c0c24b30c8..d0a6fc605f6548c5185d90ff7783ddbf84d5325e 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(); } 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 0c656d6e000421abd84da3002fcf182b053423d8..9cb3606ffb564aafdd79e122ede6af03108996b5 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 memory = new ArrayList(); for (int i = 0; i < numPages; i++) { - memory.add(new MemorySegment(new byte[pageSize])); + memory.add(MemorySegmentFactory.allocateUnpooledSegment(pageSize)); } return memory; 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 0d8b81e875ca68123cbb5bbd1d3edc5e01ec5b22..0bca22a91e6b069b2ec1874cfb1d0dffbcd038af 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 getMemory(int numSegments, int segmentSize) { ArrayList list = new ArrayList(numSegments); for (int i = 0; i < numSegments; i++) { - list.add(new MemorySegment(new byte[segmentSize])); + list.add(MemorySegmentFactory.allocateUnpooledSegment(segmentSize)); } return list; } 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 3dcf688c8999aea4ca5c75c347cf82075136a777..a71f02885e0675465613175cdb492bdecc347dbb 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 memory = new ArrayList(); for (int i = 0; i < numPages; i++) { - memory.add(new MemorySegment(new byte[pageSize])); + memory.add(MemorySegmentFactory.allocateUnpooledSegment(pageSize)); } return memory; 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 c0f8f5960f05c447937d6c2ccea35eac844769e3..70c94277b61396238b76fd139eb99daa110173e1 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(); } 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 0d5a26ef1f6b4e9953558680c83722770fb57135..2da97e9f2619e624acff5916daf6a79ea36b395e 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(); } 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 306a370c6344930f13edb21e3186cdac30b08867..86f879ac42b83a2999de6b0520cbe7354d90f7a2 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(); } 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 f770ca440eb91322efd6781c6524b8188bd0bc8a..4fdff76294e9e83754db8e90245d14f5f7ecaf88 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(); } 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 d3024874515310c60bb7f21d244500b7e6cd3a3e..ba5a32561e7da36cd9038aead8892551be5e9ba3 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(); } 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 41ade9faf55a4f9bb910564c25da711ac5ff8d4f..bfb7cf24ad9cf7dd5d3f703c1e0b506ce0a5bf94 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(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 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 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 iterator = @@ -212,5 +206,4 @@ public class BlockResettableMutableObjectIteratorTest Assert.fail("Test encountered an exception: " + ex.getMessage()); } } - } 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 5641f29c85910ca293464e045b8bac4fb18c7d7e..d5de75e164678721ff8093d90c7c948a004c3c9a 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(20000); 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 f0fadfe23b29fa87ae01b9c58f46865e47782209..7dbd2fbf446f8ea876325927774651595a0b92be 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(20000); @@ -197,5 +195,4 @@ public class ReusingBlockResettableIteratorTest // close the iterator iterator.close(); } - } 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 4db520e0189e22ed2a4b6b89aa8975b1faa8ab9f..0ab9a54bd56461937078a0a2f5bdb2f2e79499d9 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 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 fa72bbf93b6dc695e72f4024cdac8d8bb4aff783..c64db54b1e9d2d1f0dac0c96c4ffdefa6e3de6b8 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 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 d4da9d3d4ef1d843732041ee7b702c06ee675fa3..0c0e83607317a6a683d7ca787d91e5ca74693128 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> 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; 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 0ba98230f1e3756b7233a17a275aa60723e891eb..75593b82029ba7852a60e122370347b6d4550060 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 countTable = new Hashtable(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 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 46a1632d119dbabfd747fde6613a84fe18148fd5..5aa9efbe264ba9b87d477fb1fadf626491657368 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(); 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 86be6145a4a0171d2091093afdd7b096562c34fa..951ce30ce736542ef1c36b35e833f17384d1b4e5 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(); } 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 62c2013a2874b3040b756a76972bc565e90bd17b..f40171a3ee6a83ca94b8c317ada4acc871bb557f 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 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(); } 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 429062bbece96bc4bf1d1821e99c689ecb05b7d6..e4dbfd30ebeb9bd5a1aea4a4e1af270d50bb782f 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 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 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(); } } 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 423ff9af53f1e3fc327b998ac844aabbf8279d58..49e1e76ad0aa995cfa893bbb4569d4cffa68102f 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 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 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 initialMemory = memMan.allocatePages(owner, 6); @@ -264,9 +261,7 @@ public class LargeRecordHandlerTest { fail(e.getMessage()); } finally { - if (ioMan != null) { - ioMan.shutdown(); - } + ioMan.shutdown(); } } } 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 65480529831d2e06b2c6c5b59437d74975138567..7f6db009c47b331389600ad240d87201536b5bb3 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>(comparator1, comparator2); - this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1); + this.memoryManager = new MemoryManager(MEMORY_SIZE, 1); this.ioManager = new IOManagerAsync(); } 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 91609bb5b3463be6db58b3f61caf687c81cd8d73..7272595cd5d99603955c5f352f6b25c0a376b3da 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; 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 abead641b5bba8666e1ff346f6bd58407a9d1b99..3e941dd26e7d68d6ea6c8f7d11e5530d5587b5a0 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 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 memory = this.memoryManager.allocatePages(new DummyInvokable(), numSegments); 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 39316e3360fd332421f76f1374cc7747c94a1094..d92be18bd339cbb8fc50650a6d12f00a24c121d5 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>(comparator1, comparator2); - this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1); + this.memoryManager = new MemoryManager(MEMORY_SIZE, 1); this.ioManager = new IOManagerAsync(); } 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 779cf370ed55b4fc89b47841100f370218315715..2cec393eea6e55d17c2de63967020ddbf4ea45dc 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; 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 a085eeb8baf499964489728099020f6c33bab431..ece20ffd4d0b47568686b677e1ef4d082cda5ab9 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 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<>(); 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 12ca909e5ac961dd08afd46eae43c596d65b5df4..1737349d09aafb56d050cc0fb46bb8b49db2445f 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 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>(); this.comparators = new ArrayList>(); 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 5fc9bb9f081f1a81a87e53080464baac9afe70b2..972aeda18f290b54ff949a699dfe08835f6a7ef2 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(); this.outputs = new LinkedList(); - 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)); } }); 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 c04d051fb387a3e809d006dfacf82a3c2630980c..4662762fe4a23ccbe000dc5deec766b4f8627d08 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; 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 9600e5a7bce88b28f8efc6c4eee9b3ad639db460..924a16b9bffd7c0eec9ac2f0b75bf83cc7281d12 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 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(); 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 cbbeca017e14090478aaa9a85987045ec3f973cf..8fbd4f72fb622ea1324ba8b005ee46c98c36c4d8 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); } 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 7debb081c0889b9a88c29f4bc5ba9eff8bc2ccc1..1060e5561c1bb38de86600fa0f3b12d96dcd7851 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(); } 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 6d743d5eb3c4642617a52d332e1dc8e55af99b18..785712396c612029925ee0ee5acfebbc7016b1c3 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; 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 0bb40636dd4a4a9f000d83b26379e703a459f884..0a7f08d8201cb7d7aead6302ec7b1fee17d03018 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.empty(), + 32, BUFFER_SIZE, MemoryType.HEAP, IOManager.IOMode.SYNC, Option.empty(), new Tuple2(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(), 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 5cfc2a24836e60f6fe5fcbb8f6226595cccd8e16..cf1bfc5e30a14846869f7cb9445fbd5aebf03f3a 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; 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 ac6744e6b5fdee71253928becfea5b0263ae22e8..3631ba18c6392261ba1ff2348b94b43c816201c2 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 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 475115e32735bb867e8ce71ab5215d94e7aff5dd..8746fbb9a48a160f00a208c476c77b427bc18092 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, 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 2bad1979a980b3a9c1684bf1f868c77a0fe421f3..cabed14f92a815161acb1c2509b4cc3715d0856c 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; 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 173e894d1ebfb114617e65098e631cb36222d608..b78ec440ca13c405016e8c84fe194bf2ba8a025b 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 extends TestSingleInputGate { for (int i = 0; i < numInputChannels; i++) { final int channelIndex = i; final RecordSerializer> recordSerializer = new SpanningRecordSerializer>(); - final SerializationDelegate delegate = new SerializationDelegate(new MultiplexingStreamRecordSerializer(serializer)); + final SerializationDelegate delegate = new SerializationDelegate(new MultiplexingStreamRecordSerializer(serializer)); inputQueues[channelIndex] = new ConcurrentLinkedQueue>(); inputChannels[channelIndex] = new TestInputChannel(inputGate, i); @@ -98,20 +99,25 @@ public class StreamTestSingleInputGate 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); 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 678b14521cef10c4090613550d6fcb49ebd20239..a8a989bead0062c3ac79f373530157368b088f43 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 }, 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 a95839a1901c37d641b105bdd43a51e524a16f53..d4fdc595361d66c0917a9e984d677a2576642d4e 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); } 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 c6010d6863a08ecf7a518e3f05cd2fdc10c5bc35..b9b6e5f7821d664635be4f179f79d8adc6baf5cc 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); } // ------------------------------------------------------------------------ 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 355b7c80c627bfef82315fc570940d201d2d6360..e85eddb2c376866644dc89ff7b77c322d980bd6a 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); } 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 7cc19588447bbed9fe6466421d9c51a10f9d0d55..45bbbda4faf9f2c7dae015d04c60609a0245f8ee 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() { @Override public Buffer answer(InvocationOnMock invocation) { - return new Buffer(new MemorySegment(new byte[4096]), FreeingBufferRecycler.INSTANCE); + return new Buffer( + MemorySegmentFactory.allocateUnpooledSegment(4096), + FreeingBufferRecycler.INSTANCE); } }); 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 9091fa744cc38006e42ec7226a135d969b1283bb..4fec1181410798dd5b339e00b15cd3ca2aae954a 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(); this.outputs = new LinkedList(); - 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)); } }); 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 987d0bc791cc927d60cd829eb3580a5fa62e45ad..96dbeab93f9267af61e4fe7afebd8c4f9f4fda5b 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; 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 889255cc8c056e0f0768daac1bbeb7667ca310af..b61a9b67b8956cc771f86c135d2440646471d937 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(); } 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 47fbad716b954609b03a71111acfc59a684f9d72..a745177fd70397ad9811eb6dd16048a01f353500 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; 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 493de0794ed0c6645d493fe3749b15651f7854fa..cb20e72e42b346ffa22d8dbd81105631ca751a57 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 getMemory(int numSegments, int segmentSize) { ArrayList list = new ArrayList(numSegments); for (int i = 0; i < numSegments; i++) { - list.add(new MemorySegment(new byte[segmentSize])); + list.add(MemorySegmentFactory.allocateUnpooledSegment(segmentSize)); } return list; } 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 2da29e8c4787fe6648a6a14b937fe649a0f444e2..c11b93c14a28a3373c616a6d4175a4085be277f6 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 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> typeInfo = (TupleTypeInfo>) 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 bc65ab929c4006551d384d79dfc5f5a96cfb6b81..7a484e7e0ef19f511735ea7f9055808b03786b2c 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 serializer = new CopyableValueSerializer(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> typeInfo = (TupleTypeInfo>) @@ -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; 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 157aa0deac85a92a9986306ca27ddaf3515980ba..dd27eb579c70dec605e95fc1de0b5781ee618660 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, 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 21c658162915f8b8b2dbfaaaed253b6f8fca66f9..1b48612005c1cd81a2f097f9cbca5bc7073d2a2b 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]( 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 b95eb865824ce6d42217adc52f96d81cccbf2491..c9e193f234e1cf2de13821dcc1734b52ca9a9836 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, diff --git a/pom.xml b/pom.xml index cdee6f6bd6e94f2031c2ac0b26cbb4d95e032601..7f17396503e0ae7e82fbe7b03ab0e79b7324de0a 100644 --- a/pom.xml +++ b/pom.xml @@ -803,7 +803,7 @@ under the License. org.apache.maven.plugins maven-checkstyle-plugin - 2.12.1 + 2.16 validate diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml index a2864a0f58c8c2e04efd43684173164a9e26c936..ecedb1980278bbfa16c9b5c490ec2e58356eaeac 100644 --- a/tools/maven/checkstyle.xml +++ b/tools/maven/checkstyle.xml @@ -52,7 +52,7 @@ under the License. - +