diff --git a/java/flight/src/main/java/org/apache/arrow/flight/ArrowMessage.java b/java/flight/src/main/java/org/apache/arrow/flight/ArrowMessage.java index d2f7bb6c713..187437a90a5 100644 --- a/java/flight/src/main/java/org/apache/arrow/flight/ArrowMessage.java +++ b/java/flight/src/main/java/org/apache/arrow/flight/ArrowMessage.java @@ -203,7 +203,7 @@ private static ArrowMessage frame(BufferAllocator allocator, final InputStream s case BODY_TAG: if (body != null) { // only read last body. - body.release(); + body.getReferenceManager().release(); body = null; } int size = readRawVarint32(stream); @@ -271,7 +271,7 @@ private InputStream asInputStream(BufferAllocator allocator) { int size = 0; List allBufs = new ArrayList<>(); for (ArrowBuf b : bufs) { - allBufs.add(b); + allBufs.add(b.asNettyBuffer()); size += b.readableBytes(); // [ARROW-4213] These buffers must be aligned to an 8-byte boundary in order to be readable from C++. if (b.readableBytes() % 8 != 0) { @@ -288,7 +288,7 @@ private InputStream asInputStream(BufferAllocator allocator) { ArrowBuf initialBuf = allocator.buffer(baos.size()); initialBuf.writeBytes(baos.toByteArray()); final CompositeByteBuf bb = new CompositeByteBuf(allocator.getAsByteBufAllocator(), true, bufs.size() + 1, - ImmutableList.builder().add(initialBuf).addAll(allBufs).build()); + ImmutableList.builder().add(initialBuf.asNettyBuffer()).addAll(allBufs).build()); final ByteBufInputStream is = new DrainableByteBufInputStream(bb); return is; } catch (Exception ex) { diff --git a/java/gandiva/src/test/java/org/apache/arrow/gandiva/evaluator/BaseEvaluatorTest.java b/java/gandiva/src/test/java/org/apache/arrow/gandiva/evaluator/BaseEvaluatorTest.java index 2fd5a246329..2f44855ead6 100644 --- a/java/gandiva/src/test/java/org/apache/arrow/gandiva/evaluator/BaseEvaluatorTest.java +++ b/java/gandiva/src/test/java/org/apache/arrow/gandiva/evaluator/BaseEvaluatorTest.java @@ -278,7 +278,7 @@ void releaseRecordBatch(ArrowRecordBatch recordBatch) { List buffers = recordBatch.getBuffers(); recordBatch.close(); for (ArrowBuf buf : buffers) { - buf.release(); + buf.getReferenceManager().release(); } } diff --git a/java/memory/src/main/java/io/netty/buffer/ArrowBuf.java b/java/memory/src/main/java/io/netty/buffer/ArrowBuf.java index bb6a940847f..ae7d354b677 100644 --- a/java/memory/src/main/java/io/netty/buffer/ArrowBuf.java +++ b/java/memory/src/main/java/io/netty/buffer/ArrowBuf.java @@ -22,150 +22,90 @@ import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.FileChannel; -import java.nio.channels.GatheringByteChannel; -import java.nio.channels.ScatteringByteChannel; -import java.nio.charset.Charset; -import java.util.concurrent.atomic.AtomicInteger; +import java.nio.ReadOnlyBufferException; import java.util.concurrent.atomic.AtomicLong; -import org.apache.arrow.memory.AllocationManager.BufferLedger; -import org.apache.arrow.memory.ArrowByteBufAllocator; +import org.apache.arrow.memory.AllocationManager; import org.apache.arrow.memory.BaseAllocator; import org.apache.arrow.memory.BaseAllocator.Verbosity; import org.apache.arrow.memory.BoundsChecking; -import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.BufferLedger; import org.apache.arrow.memory.BufferManager; +import org.apache.arrow.memory.ReferenceManager; import org.apache.arrow.memory.util.HistoricalLog; import org.apache.arrow.util.Preconditions; import io.netty.util.internal.PlatformDependent; /** - * ArrowBuf is the abstraction around raw byte arrays that - * comprise arrow data structures. - * - * - *

Specifically, it serves as a facade over - * {@linkplain UnsafeDirectLittleEndian} memory objects that hides the details - * of raw memory addresses. - * - *

ArrowBuf supports reference counting and ledgering to closely track where - * memory is being used. + * ArrowBuf serves as a facade over underlying memory by providing + * several access APIs to read/write data into a chunk of direct + * memory. All the accounting, ownership and reference management + * is done by {@link ReferenceManager} and ArrowBuf can work + * with a custom user provided implementation of ReferenceManager + *

+ * Two important instance variables of an ArrowBuf: + * (1) address - starting virtual address in the underlying memory + * chunk that this ArrowBuf has access to + * (2) length - length (in bytes) in the underlying memory chunk + * that this ArrowBuf has access to + *

+ *

+ * The mangement (allocation, deallocation, reference counting etc) for + * the memory chunk is not done by ArrowBuf. + * Default implementation of ReferenceManager, allocation is in + * {@link BaseAllocator}, {@link BufferLedger} and {@link AllocationManager} + *

*/ -public final class ArrowBuf extends AbstractByteBuf implements AutoCloseable { +public final class ArrowBuf implements AutoCloseable { private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ArrowBuf.class); + private static final int SHORT_SIZE = Short.BYTES; + private static final int INT_SIZE = Integer.BYTES; + private static final int FLOAT_SIZE = Float.BYTES; + private static final int DOUBLE_SIZE = Double.BYTES; + private static final int LONG_SIZE = Long.BYTES; + private static final AtomicLong idGenerator = new AtomicLong(0); private static final int LOG_BYTES_PER_ROW = 10; private final long id = idGenerator.incrementAndGet(); - private final AtomicInteger refCnt; - private final UnsafeDirectLittleEndian udle; + private final ReferenceManager referenceManager; + private final BufferManager bufferManager; private final long addr; - private final int offset; - private final BufferLedger ledger; - private final BufferManager bufManager; - private final ArrowByteBufAllocator alloc; private final boolean isEmpty; + private int readerIndex; + private int writerIndex; private final HistoricalLog historicalLog = BaseAllocator.DEBUG ? new HistoricalLog(BaseAllocator.DEBUG_LOG_LENGTH, "ArrowBuf[%d]", id) : null; private volatile int length; /** * Constructs a new ArrowBuf - * @param refCnt The atomic integer to use for reference counting this buffer. - * @param ledger The ledger to use for tracking memory usage of this buffer. - * @param byteBuf The underlying storage for this buffer. - * @param manager The manager that handles replacing this buffer. - * @param alloc The allocator for the buffer (needed for superclass compatibility) - * @param offset The byte offset into byteBuf this buffer starts at. + * @param referenceManager The memory manager to track memory usage and reference count of this buffer * @param length The byte length of this buffer * @param isEmpty Indicates if this buffer is empty which enables some optimizations. */ public ArrowBuf( - final AtomicInteger refCnt, - final BufferLedger ledger, - final UnsafeDirectLittleEndian byteBuf, - final BufferManager manager, - final ArrowByteBufAllocator alloc, - final int offset, + final ReferenceManager referenceManager, + final BufferManager bufferManager, final int length, + final long memoryAddress, boolean isEmpty) { - // TODO(emkornfield): Should this be byteBuf.maxCapacity - offset? - super(byteBuf.maxCapacity()); - this.refCnt = refCnt; - this.udle = byteBuf; + this.referenceManager = referenceManager; + this.bufferManager = bufferManager; this.isEmpty = isEmpty; - this.bufManager = manager; - this.alloc = alloc; - this.addr = byteBuf.memoryAddress() + offset; - this.ledger = ledger; + this.addr = memoryAddress; this.length = length; - this.offset = offset; - + this.readerIndex = 0; + this.writerIndex = 0; if (BaseAllocator.DEBUG) { historicalLog.recordEvent("create()"); } - - } - - /** Returns a debug friendly string for the given ByteBuf. */ - public static String bufferState(final ByteBuf buf) { - final int cap = buf.capacity(); - final int mcap = buf.maxCapacity(); - final int ri = buf.readerIndex(); - final int rb = buf.readableBytes(); - final int wi = buf.writerIndex(); - final int wb = buf.writableBytes(); - return String.format("cap/max: %d/%d, ri: %d, rb: %d, wi: %d, wb: %d", - cap, mcap, ri, rb, wi, wb); } - /** - * Returns this if size is less then {@link #capacity()}, otherwise - * delegates to {@link BufferManager#replace(ArrowBuf, int)} to get a new buffer. - */ - public ArrowBuf reallocIfNeeded(final int size) { - Preconditions.checkArgument(size >= 0, "reallocation size must be non-negative"); - - if (this.capacity() >= size) { - return this; - } - - if (bufManager != null) { - return bufManager.replace(this, size); - } else { - throw new UnsupportedOperationException("Realloc is only available in the context of an " + - "operator's UDFs"); - } - } - - @Override public int refCnt() { - if (isEmpty) { - return 1; - } else { - return refCnt.get(); - } - } - - private long addr(int index) { - return addr + index; - } - - private final void checkIndexD(int index, int fieldLength) { - ensureAccessible(); - if (fieldLength < 0) { - throw new IllegalArgumentException("length: " + fieldLength + " (expected: >= 0)"); - } - if (index < 0 || index > capacity() - fieldLength) { - if (BaseAllocator.DEBUG) { - historicalLog.logHistory(logger); - } - throw new IndexOutOfBoundsException(String.format( - "index: %d, length: %d (expected: range(0, %d))", index, fieldLength, capacity())); - } + return isEmpty ? 1 : referenceManager.getRefCount(); } /** @@ -184,142 +124,47 @@ public void checkBytes(int start, int end) { } } - private void chk(int index, int width) { - if (BoundsChecking.BOUNDS_CHECKING_ENABLED) { - checkIndexD(index, width); - } - } - - private void ensure(int width) { - if (BoundsChecking.BOUNDS_CHECKING_ENABLED) { - ensureWritable(width); - } - } - /** - * Create a new ArrowBuf that is associated with an alternative allocator for the purposes of - * memory ownership and - * accounting. This has no impact on the reference counting for the current ArrowBuf except in - * the situation where the - * passed in Allocator is the same as the current buffer. - * - *

This operation has no impact on the reference count of this ArrowBuf. The newly created - * ArrowBuf with either have a - * reference count of 1 (in the case that this is the first time this memory is being - * associated with the new - * allocator) or the current value of the reference count + 1 for the other - * AllocationManager/BufferLedger combination - * in the case that the provided allocator already had an association to this underlying memory. - * - * @param target The target allocator to create an association with. - * @return A new ArrowBuf which shares the same underlying memory as this ArrowBuf. + * For get/set operations, reference count should be >= 1. */ - public ArrowBuf retain(BufferAllocator target) { - - if (isEmpty) { - return this; + private void ensureAccessible() { + if (this.refCnt() == 0) { + throw new IllegalStateException("Ref count should be >= 1 for accessing the ArrowBuf"); } - - if (BaseAllocator.DEBUG) { - historicalLog.recordEvent("retain(%s)", target.getName()); - } - final BufferLedger otherLedger = this.ledger.getLedgerForAllocator(target); - ArrowBuf newArrowBuf = otherLedger.newArrowBuf(offset, length, null); - newArrowBuf.readerIndex(this.readerIndex); - newArrowBuf.writerIndex(this.writerIndex); - return newArrowBuf; } /** - * Transfer the memory accounting ownership of this ArrowBuf to another allocator. This will - * generate a new ArrowBuf - * that carries an association with the underlying memory of this ArrowBuf. If this ArrowBuf is - * connected to the - * owning BufferLedger of this memory, that memory ownership/accounting will be transferred to - * the target allocator. If - * this ArrowBuf does not currently own the memory underlying it (and is only associated with - * it), this does not - * transfer any ownership to the newly created ArrowBuf. - * - *

This operation has no impact on the reference count of this ArrowBuf. The newly created - * ArrowBuf with either have a - * reference count of 1 (in the case that this is the first time this memory is being - * associated with the new - * allocator) or the current value of the reference count for the other - * AllocationManager/BufferLedger combination in - * the case that the provided allocator already had an association to this underlying memory. - * - *

Transfers will always succeed, even if that puts the other allocator into an overlimit - * situation. This is possible - * due to the fact that the original owning allocator may have allocated this memory out of a - * local reservation - * whereas the target allocator may need to allocate new memory from a parent or RootAllocator. - * This operation is done - * in a mostly-lockless but consistent manner. As such, the overlimit==true situation could - * occur slightly prematurely - * to an actual overlimit==true condition. This is simply conservative behavior which means we - * may return overlimit - * slightly sooner than is necessary. - * - * @param target The allocator to transfer ownership to. - * @return A new transfer result with the impact of the transfer (whether it was overlimit) as - * well as the newly created ArrowBuf. + * Get a wrapper buffer to comply with Netty interfaces and + * can be used in RPC/RPC allocator code. + * @return netty compliant {@link NettyArrowBuf} */ - public TransferResult transferOwnership(BufferAllocator target) { + public NettyArrowBuf asNettyBuffer() { - if (isEmpty) { - return new TransferResult(true, this); - } - - final BufferLedger otherLedger = this.ledger.getLedgerForAllocator(target); - final ArrowBuf newBuf = otherLedger.newArrowBuf(offset, length, null); - newBuf.setIndex(this.readerIndex, this.writerIndex); - final boolean allocationFit = this.ledger.transferBalance(otherLedger); - return new TransferResult(allocationFit, newBuf); - } - - @Override - public boolean release() { - return release(1); + final NettyArrowBuf nettyArrowBuf = new NettyArrowBuf( + this, + isEmpty ? null : referenceManager.getAllocator().getAsByteBufAllocator(), + length); + nettyArrowBuf.readerIndex(readerIndex); + nettyArrowBuf.writerIndex(writerIndex); + return nettyArrowBuf; } /** - * Release the provided number of reference counts. + * Get reference manager for this ArrowBuf. + * @return user provided implementation of {@link ReferenceManager} */ - @Override - public boolean release(int decrement) { - - if (isEmpty) { - return false; - } - - if (decrement < 1) { - throw new IllegalStateException(String.format("release(%d) argument is not positive. Buffer Info: %s", - decrement, toVerboseString())); - } - - final int refCnt = ledger.decrement(decrement); - - if (BaseAllocator.DEBUG) { - historicalLog.recordEvent("release(%d). original value: %d", decrement, refCnt + decrement); - } - - if (refCnt < 0) { - throw new IllegalStateException( - String.format("ArrowBuf[%d] refCnt has gone negative. Buffer Info: %s", id, - toVerboseString())); - } - - return refCnt == 0; + public ReferenceManager getReferenceManager() { + return referenceManager; + } + public boolean isEmpty() { + return isEmpty; } - @Override public int capacity() { return length; } - @Override public synchronized ArrowBuf capacity(int newCapacity) { if (newCapacity == length) { @@ -336,61 +181,25 @@ public synchronized ArrowBuf capacity(int newCapacity) { throw new UnsupportedOperationException("Buffers don't support resizing that increases the size."); } - @Override - public ArrowByteBufAllocator alloc() { - return alloc; - } - - @Override public ByteOrder order() { return ByteOrder.LITTLE_ENDIAN; } - @Override - public ArrowBuf order(ByteOrder endianness) { - return this; - } - - @Override - public ByteBuf unwrap() { - return udle; - } - - @Override - public boolean isDirect() { - return true; - } - - @Override - public ByteBuf readBytes(int length) { - throw new UnsupportedOperationException(); + public int readableBytes() { + Preconditions.checkState(writerIndex >= readerIndex, + "Writer index cannot be less than reader index"); + return writerIndex - readerIndex; } - @Override - public ByteBuf readSlice(int length) { - final ByteBuf slice = slice(readerIndex(), length); - readerIndex(readerIndex() + length); - return slice; + public int writableBytes() { + return capacity() - writerIndex; } - @Override - public ByteBuf copy() { - throw new UnsupportedOperationException(); - } - - @Override - public ByteBuf copy(int index, int length) { - throw new UnsupportedOperationException(); - } - - @Override public ArrowBuf slice() { - return slice(readerIndex(), readableBytes()); + return slice(readerIndex, readableBytes()); } - @Override public ArrowBuf slice(int index, int length) { - if (isEmpty) { return this; } @@ -403,89 +212,26 @@ public ArrowBuf slice(int index, int length) { * .html#wiki-h3-5, which * explains that derived buffers share their reference count with their parent */ - final ArrowBuf newBuf = ledger.newArrowBuf(offset + index, length); + final ArrowBuf newBuf = referenceManager.deriveBuffer(this, index, length); newBuf.writerIndex(length); return newBuf; } - @Override - public ArrowBuf duplicate() { - return slice(0, length); - } - - @Override - public int nioBufferCount() { - return 1; - } - - @Override public ByteBuffer nioBuffer() { - return nioBuffer(readerIndex(), readableBytes()); + return isEmpty ? ByteBuffer.allocateDirect(0) : asNettyBuffer().nioBuffer(); } - @Override public ByteBuffer nioBuffer(int index, int length) { - return udle.nioBuffer(offset + index, length); - } - - @Override - public ByteBuffer internalNioBuffer(int index, int length) { - return udle.internalNioBuffer(offset + index, length); - } - - @Override - public ByteBuffer[] nioBuffers() { - return new ByteBuffer[] {nioBuffer()}; + return isEmpty ? ByteBuffer.allocateDirect(0) : asNettyBuffer().nioBuffer(index, length); } - @Override - public ByteBuffer[] nioBuffers(int index, int length) { - return new ByteBuffer[] {nioBuffer(index, length)}; - } - - @Override - public boolean hasArray() { - return udle.hasArray(); - } - - @Override - public byte[] array() { - return udle.array(); - } - - @Override - public int arrayOffset() { - return udle.arrayOffset(); - } - - @Override - public boolean hasMemoryAddress() { - return true; - } - - @Override public long memoryAddress() { return this.addr; } @Override public String toString() { - return String.format("ArrowBuf[%d], udle: [%d %d..%d]", id, udle.id, offset, offset + capacity()); - } - - @Override - public String toString(Charset charset) { - return toString(readerIndex, readableBytes(), charset); - } - - @Override - public String toString(int index, int length, Charset charset) { - - if (length == 0) { - return ""; - } - - return ByteBufUtil.decodeString(this, index, length, charset); + return String.format("ArrowBuf[%d], address:%d, length:%d", id, memoryAddress(), length); } @Override @@ -499,552 +245,800 @@ public boolean equals(Object obj) { return this == obj; } - @Override - public ArrowBuf retain(int increment) { - Preconditions.checkArgument(increment > 0, "retain(%d) argument is not positive", increment); - - if (isEmpty) { - return this; - } - - if (BaseAllocator.DEBUG) { - historicalLog.recordEvent("retain(%d)", increment); - } - - final int originalReferenceCount = refCnt.getAndAdd(increment); - Preconditions.checkArgument(originalReferenceCount > 0); - return this; + /* + * IMPORTANT NOTE + * The data getters and setters work with a caller provided + * index. This index is 0 based and since ArrowBuf has access + * to a portion of underlying chunk of memory starting at + * some address, we convert the given relative index into + * absolute index as memory address + index. + * + * Example: + * + * Let's say we have an underlying chunk of memory of length 64 bytes + * Now let's say we have an ArrowBuf that has access to the chunk + * from offset 4 for length of 16 bytes. + * + * If the starting virtual address of chunk is MAR, then memory + * address of this ArrowBuf is MAR + offset -- this is what is stored + * in variable addr. See the BufferLedger and AllocationManager code + * for the implementation of ReferenceManager that manages a + * chunk of memory and creates ArrowBuf with access to a range of + * bytes within the chunk (or the entire chunk) + * + * So now to get/set data, we will do => addr + index + * This logic is put in method addr(index) and is frequently + * used in get/set data methods to compute the absolute + * byte address for get/set operation in the underlying chunk + * + * @param index the index at which we the user wants to read/write + * @return the absolute address within the memro + */ + private long addr(int index) { + return addr + index; } - @Override - public ArrowBuf retain() { - return retain(1); - } - @Override - public ByteBuf touch() { - return this; - } - @Override - public ByteBuf touch(Object hint) { - return this; - } + /*-------------------------------------------------* + | Following are a set of fast path data set and | + | get APIs to write/read data from ArrowBuf | + | at a given index (0 based relative to this | + | ArrowBuf and not relative to the underlying | + | memory chunk). | + | | + *-------------------------------------------------*/ - @Override - public long getLong(int index) { - chk(index, 8); - final long v = PlatformDependent.getLong(addr(index)); - return v; - } - @Override - public float getFloat(int index) { - return Float.intBitsToFloat(getInt(index)); - } /** - * Gets a 64-bit long integer at the specified absolute {@code index} in - * this buffer in Big Endian Byte Order. + * Helper function to do bounds checking at a particular + * index for particular length of data. + * @param index index (0 based relative to this ArrowBuf) + * @param length provided length of data for get/set */ - @Override - public long getLongLE(int index) { - chk(index, 8); - final long v = PlatformDependent.getLong(addr(index)); - return Long.reverseBytes(v); - } - - @Override - public double getDouble(int index) { - return Double.longBitsToDouble(getLong(index)); - } - - @Override - public char getChar(int index) { - return (char) getShort(index); - } - - @Override - public long getUnsignedInt(int index) { - return getInt(index) & 0xFFFFFFFFL; + private void chk(int index, int length) { + if (BoundsChecking.BOUNDS_CHECKING_ENABLED) { + checkIndexD(index, length); + } } - @Override - public int getInt(int index) { - chk(index, 4); - final int v = PlatformDependent.getInt(addr(index)); - return v; + private void checkIndexD(int index, int fieldLength) { + // check reference count + ensureAccessible(); + // check bounds + Preconditions.checkArgument(fieldLength >= 0, "expecting non-negative data length"); + if (index < 0 || index > capacity() - fieldLength) { + if (BaseAllocator.DEBUG) { + historicalLog.logHistory(logger); + } + throw new IndexOutOfBoundsException(String.format( + "index: %d, length: %d (expected: range(0, %d))", index, fieldLength, capacity())); + } } /** - * Gets a 32-bit integer at the specified absolute {@code index} in - * this buffer in Big Endian Byte Order. + * Get long value stored at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be read from + * @return 8 byte long value */ - @Override - public int getIntLE(int index) { - chk(index, 4); - final int v = PlatformDependent.getInt(addr(index)); - return Integer.reverseBytes(v); - } - - @Override - public int getUnsignedShort(int index) { - return getShort(index) & 0xFFFF; + public long getLong(int index) { + chk(index, LONG_SIZE); + return PlatformDependent.getLong(addr(index)); } - @Override - public short getShort(int index) { - chk(index, 2); - final short v = PlatformDependent.getShort(addr(index)); - return v; + /** + * Set long value at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be written + * @param value value to write + */ + public void setLong(int index, long value) { + chk(index, LONG_SIZE); + PlatformDependent.putLong(addr(index), value); } /** - * Gets a 16-bit short integer at the specified absolute {@code index} in - * this buffer in Big Endian Byte Order. + * Get float value stored at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be read from + * @return 4 byte float value */ - @Override - public short getShortLE(int index) { - final short v = PlatformDependent.getShort(addr(index)); - return Short.reverseBytes(v); + public float getFloat(int index) { + return Float.intBitsToFloat(getInt(index)); } /** - * Gets an unsigned 24-bit medium integer at the specified absolute - * {@code index} in this buffer. + * Set float value at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be written + * @param value value to write */ - @Override - public int getUnsignedMedium(int index) { - chk(index, 3); - final long addr = addr(index); - return (PlatformDependent.getByte(addr) & 0xff) << 16 | - (PlatformDependent.getShort(addr + 1) & 0xffff); + public void setFloat(int index, float value) { + chk(index, FLOAT_SIZE); + PlatformDependent.putInt(addr(index), Float.floatToRawIntBits(value)); } /** - * Gets an unsigned 24-bit medium integer at the specified absolute {@code index} in - * this buffer in Big Endian Byte Order. + * Get double value stored at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be read from + * @return 8 byte double value */ - @Override - public int getUnsignedMediumLE(int index) { - chk(index, 3); - final long addr = addr(index); - return (PlatformDependent.getByte(addr) & 0xff) | - (Short.reverseBytes(PlatformDependent.getShort(addr + 1)) & 0xffff) << 8; + public double getDouble(int index) { + return Double.longBitsToDouble(getLong(index)); } - @Override - public ArrowBuf setShort(int index, int value) { - chk(index, 2); - PlatformDependent.putShort(addr(index), (short) value); - return this; + /** + * Set double value at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be written + * @param value value to write + */ + public void setDouble(int index, double value) { + chk(index, DOUBLE_SIZE); + PlatformDependent.putLong(addr(index), Double.doubleToRawLongBits(value)); } /** - * Sets the specified 16-bit short integer at the specified absolute {@code index} - * in this buffer with Big Endian byte order. + * Get char value stored at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be read from + * @return 2 byte char value */ - @Override - public ByteBuf setShortLE(int index, int value) { - chk(index, 2); - PlatformDependent.putShort(addr(index), Short.reverseBytes((short) value)); - return this; + public char getChar(int index) { + return (char) getShort(index); } /** - * Sets the specified 24-bit medium integer at the specified absolute - * {@code index} in this buffer. + * Set char value at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be written + * @param value value to write */ - @Override - public ByteBuf setMedium(int index, int value) { - chk(index, 3); - final long addr = addr(index); - PlatformDependent.putByte(addr, (byte) (value >>> 16)); - PlatformDependent.putShort(addr + 1, (short) value); - return this; + public void setChar(int index, int value) { + chk(index, SHORT_SIZE); + PlatformDependent.putShort(addr(index), (short) value); } - /** - * Sets the specified 24-bit medium integer at the specified absolute {@code index} - * in this buffer with Big Endian byte order. + * Get int value stored at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be read from + * @return 4 byte int value */ - @Override - public ByteBuf setMediumLE(int index, int value) { - chk(index, 3); - final long addr = addr(index); - PlatformDependent.putByte(addr, (byte) value); - PlatformDependent.putShort(addr + 1, Short.reverseBytes((short) (value >>> 8))); - return this; + public int getInt(int index) { + chk(index, INT_SIZE); + return PlatformDependent.getInt(addr(index)); } - @Override - public ArrowBuf setInt(int index, int value) { - chk(index, 4); + /** + * Set int value at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be written + * @param value value to write + */ + public void setInt(int index, int value) { + chk(index, INT_SIZE); PlatformDependent.putInt(addr(index), value); - return this; } /** - * Sets the specified 32-bit integer at the specified absolute {@code index} - * in this buffer with Big Endian byte order. + * Get short value stored at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be read from + * @return 2 byte short value */ - @Override - public ByteBuf setIntLE(int index, int value) { - chk(index, 4); - PlatformDependent.putInt(addr(index), Integer.reverseBytes(value)); - return this; - } - - @Override - public ArrowBuf setLong(int index, long value) { - chk(index, 8); - PlatformDependent.putLong(addr(index), value); - return this; + public short getShort(int index) { + chk(index, SHORT_SIZE); + return PlatformDependent.getShort(addr(index)); } /** - * Sets the specified 64-bit long integer at the specified absolute {@code index} - * in this buffer with Big Endian byte order. + * Set short value at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be written + * @param value value to write */ - @Override - public ByteBuf setLongLE(int index, long value) { - chk(index, 8); - PlatformDependent.putLong(addr(index), Long.reverseBytes(value)); - return this; + public void setShort(int index, int value) { + setShort(index, (short)value); } - @Override - public ArrowBuf setChar(int index, int value) { - chk(index, 2); - PlatformDependent.putShort(addr(index), (short) value); - return this; + /** + * Set short value at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be written + * @param value value to write + */ + public void setShort(int index, short value) { + chk(index, SHORT_SIZE); + PlatformDependent.putShort(addr(index), value); } - @Override - public ArrowBuf setFloat(int index, float value) { - chk(index, 4); - PlatformDependent.putInt(addr(index), Float.floatToRawIntBits(value)); - return this; + /** + * Set byte value at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be written + * @param value value to write + */ + public void setByte(int index, int value) { + chk(index, 1); + PlatformDependent.putByte(addr(index), (byte) value); } - @Override - public ArrowBuf setDouble(int index, double value) { - chk(index, 8); - PlatformDependent.putLong(addr(index), Double.doubleToRawLongBits(value)); - return this; + /** + * Set byte value at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be written + * @param value value to write + */ + public void setByte(int index, byte value) { + chk(index, 1); + PlatformDependent.putByte(addr(index), value); } - @Override - public ArrowBuf writeShort(int value) { - ensure(2); - PlatformDependent.putShort(addr(writerIndex), (short) value); - writerIndex += 2; - return this; + /** + * Get byte value stored at a particular index in the + * underlying memory chunk this ArrowBuf has access to. + * @param index index (0 based relative to this ArrowBuf) + * where the value will be read from + * @return byte value + */ + public byte getByte(int index) { + chk(index, 1); + return PlatformDependent.getByte(addr(index)); } - @Override - public ArrowBuf writeInt(int value) { - ensure(4); - PlatformDependent.putInt(addr(writerIndex), value); - writerIndex += 4; - return this; - } - @Override - public ArrowBuf writeLong(long value) { - ensure(8); - PlatformDependent.putLong(addr(writerIndex), value); - writerIndex += 8; - return this; - } - @Override - public ArrowBuf writeChar(int value) { - ensure(2); - PlatformDependent.putShort(addr(writerIndex), (short) value); - writerIndex += 2; - return this; - } + /*--------------------------------------------------* + | Following are another set of data set APIs | + | that directly work with writerIndex | + | | + *--------------------------------------------------*/ - @Override - public ArrowBuf writeFloat(float value) { - ensure(4); - PlatformDependent.putInt(addr(writerIndex), Float.floatToRawIntBits(value)); - writerIndex += 4; - return this; - } - @Override - public ArrowBuf writeDouble(double value) { - ensure(8); - PlatformDependent.putLong(addr(writerIndex), Double.doubleToRawLongBits(value)); - writerIndex += 8; - return this; - } - @Override - public ArrowBuf getBytes(int index, byte[] dst, int dstIndex, int length) { - udle.getBytes(index + offset, dst, dstIndex, length); - return this; - } - - @Override - public ArrowBuf getBytes(int index, ByteBuffer dst) { - udle.getBytes(index + offset, dst); - return this; + /** + * Helper function to do bound checking w.r.t writerIndex + * by checking if we can set "length" bytes of data at the + * writerIndex in this ArrowBuf. + * @param length provided length of data for set + */ + private void ensureWritable(final int length) { + if (BoundsChecking.BOUNDS_CHECKING_ENABLED) { + Preconditions.checkArgument(length >= 0, "expecting non-negative length"); + // check reference count + this.ensureAccessible(); + // check bounds + if (length > writableBytes()) { + throw new IndexOutOfBoundsException( + String.format("writerIndex(%d) + length(%d) exceeds capacity(%d)", writerIndex, length, capacity())); + } + } } - @Override - public ArrowBuf setByte(int index, int value) { - chk(index, 1); - PlatformDependent.putByte(addr(index), (byte) value); - return this; + /** + * Helper function to do bound checking w.r.t readerIndex + * by checking if we can read "length" bytes of data at the + * readerIndex in this ArrowBuf. + * @param length provided length of data for get + */ + private void ensureReadable(final int length) { + if (BoundsChecking.BOUNDS_CHECKING_ENABLED) { + Preconditions.checkArgument(length >= 0, "expecting non-negative length"); + // check reference count + this.ensureAccessible(); + // check bounds + if (length > readableBytes()) { + throw new IndexOutOfBoundsException( + String.format("readerIndex(%d) + length(%d) exceeds writerIndex(%d)", readerIndex, length, writerIndex)); + } + } } - public void setByte(int index, byte b) { - chk(index, 1); - PlatformDependent.putByte(addr(index), b); + /** + * Read the byte at readerIndex. + * @return byte value + */ + public byte readByte() { + ensureReadable(1); + final byte b = getByte(readerIndex); + ++readerIndex; + return b; } - public void writeByteUnsafe(byte b) { - PlatformDependent.putByte(addr(readerIndex), b); - readerIndex++; + /** + * Read dst.length bytes at readerIndex into dst byte array + * @param dst byte array where the data will be written + */ + public void readBytes(byte[] dst) { + Preconditions.checkArgument(dst != null, "expecting valid dst bytearray"); + ensureReadable(dst.length); + getBytes(readerIndex, dst, 0, dst.length); } - @Override - protected byte _getByte(int index) { - return getByte(index); + /** + * Set the provided byte value at the writerIndex. + * @param value value to set + */ + public void writeByte(byte value) { + ensureWritable(1); + PlatformDependent.putByte(addr(writerIndex), value); + ++writerIndex; } - @Override - protected short _getShort(int index) { - return getShort(index); + /** + * Set the lower order byte for the provided value at + * the writerIndex. + * @param value value to be set + */ + public void writeByte(int value) { + ensureWritable(1); + PlatformDependent.putByte(addr(writerIndex), (byte)value); + ++writerIndex; } /** - * @see ArrowBuf#getShortLE(int). + * Write the bytes from given byte array into this + * ArrowBuf starting at writerIndex. + * @param src src byte array */ - @Override - protected short _getShortLE(int index) { - return getShortLE(index); + public void writeBytes(byte[] src) { + Preconditions.checkArgument(src != null, "expecting valid src array"); + writeBytes(src, 0, src.length); } - @Override - protected int _getInt(int index) { - return getInt(index); + /** + * Write the bytes from given byte array starting at srcIndex + * into this ArrowBuf starting at writerIndex. + * @param src src byte array + * @param srcIndex index in the byte array where the copy will being from + * @param length length of data to copy + */ + public void writeBytes(byte[] src, int srcIndex, int length) { + ensureWritable(length); + setBytes(writerIndex, src, srcIndex, length); + writerIndex += length; } /** - * @see ArrowBuf#getIntLE(int). + * Set the provided int value as short at the writerIndex. + * @param value value to set */ - @Override - protected int _getIntLE(int index) { - return getIntLE(index); + public void writeShort(int value) { + ensureWritable(SHORT_SIZE); + PlatformDependent.putShort(addr(writerIndex), (short) value); + writerIndex += SHORT_SIZE; } /** - * @see ArrowBuf#getUnsignedMedium(int). + * Set the provided int value at the writerIndex. + * @param value value to set */ - @Override - protected int _getUnsignedMedium(int index) { - return getUnsignedMedium(index); + public void writeInt(int value) { + ensureWritable(INT_SIZE); + PlatformDependent.putInt(addr(writerIndex), value); + writerIndex += INT_SIZE; } /** - * @see ArrowBuf#getUnsignedMediumLE(int). + * Set the provided long value at the writerIndex. + * @param value value to set */ - @Override - protected int _getUnsignedMediumLE(int index) { - return getUnsignedMediumLE(index); + public void writeLong(long value) { + ensureWritable(LONG_SIZE); + PlatformDependent.putLong(addr(writerIndex), value); + writerIndex += LONG_SIZE; } - @Override - protected long _getLong(int index) { - return getLong(index); + /** + * Set the provided float value at the writerIndex. + * @param value value to set + */ + public void writeFloat(float value) { + ensureWritable(FLOAT_SIZE); + PlatformDependent.putInt(addr(writerIndex), Float.floatToRawIntBits(value)); + writerIndex += FLOAT_SIZE; } /** - * @see ArrowBuf#getLongLE(int). + * Set the provided double value at the writerIndex. + * @param value value to set */ - @Override - protected long _getLongLE(int index) { - return getLongLE(index); + public void writeDouble(double value) { + ensureWritable(DOUBLE_SIZE); + PlatformDependent.putLong(addr(writerIndex), Double.doubleToRawLongBits(value)); + writerIndex += DOUBLE_SIZE; } - @Override - protected void _setByte(int index, int value) { - setByte(index, value); - } - @Override - protected void _setShort(int index, int value) { - setShort(index, value); - } + /*--------------------------------------------------* + | Following are another set of data set/get APIs | + | that read and write stream of bytes from/to byte | + | arrays, ByteBuffer, ArrowBuf etc | + | | + *--------------------------------------------------*/ /** - * @see ArrowBuf#setShortLE(int, int). + * Determine if the requested {@code index} and {@code length} will fit within {@code capacity}. + * @param index The starting index. + * @param length The length which will be utilized (starting from {@code index}). + * @param capacity The capacity that {@code index + length} is allowed to be within. + * @return {@code true} if the requested {@code index} and {@code length} will fit within {@code capacity}. + * {@code false} if this would result in an index out of bounds exception. */ - @Override - protected void _setShortLE(int index, int value) { - setShortLE(index, value); + private static boolean isOutOfBounds(int index, int length, int capacity) { + return (index | length | (index + length) | (capacity - (index + length))) < 0; } - @Override - protected void _setMedium(int index, int value) { - setMedium(index, value); + private void checkIndex(int index, int fieldLength) { + // check reference count + this.ensureAccessible(); + // check bounds + if (isOutOfBounds(index, fieldLength, this.capacity())) { + throw new IndexOutOfBoundsException(String.format("index: %d, length: %d (expected: range(0, %d))", + index, fieldLength, this.capacity())); + } } /** - * @see ArrowBuf#setMediumLE(int, int). + * Copy data from this ArrowBuf at a given index in into destination + * byte array. + * @param index starting index (0 based relative to the portion of memory) + * this ArrowBuf has access to + * @param dst byte array to copy the data into */ - @Override - protected void _setMediumLE(int index, int value) { - setMediumLE(index, value); - } - - @Override - protected void _setInt(int index, int value) { - setInt(index, value); + public void getBytes(int index, byte[] dst) { + getBytes(index, dst, 0, dst.length); } /** - * @see ArrowBuf#setIntLE(int, int). + * Copy data from this ArrowBuf at a given index into destination byte array. + * @param index index (0 based relative to the portion of memory + * this ArrowBuf has access to) + * @param dst byte array to copy the data into + * @param dstIndex starting index in dst byte array to copy into + * @param length length of data to copy from this ArrowBuf */ - @Override - protected void _setIntLE(int index, int value) { - setIntLE(index, value); - } - - @Override - protected void _setLong(int index, long value) { - setLong(index, value); + public void getBytes(int index, byte[] dst, int dstIndex, int length) { + // bound check for this ArrowBuf where the data will be copied from + checkIndex(index, length); + // null check + Preconditions.checkArgument(dst != null, "expecting a valid dst byte array"); + // bound check for dst byte array where the data will be copied to + if (isOutOfBounds(dstIndex, length, dst.length)) { + // not enough space to copy "length" bytes into dst array from dstIndex onwards + throw new IndexOutOfBoundsException("Not enough space to copy data into destination" + dstIndex); + } + if (length != 0) { + // copy "length" bytes from this ArrowBuf starting at addr(index) address + // into dst byte array at dstIndex onwards + PlatformDependent.copyMemory(addr(index), dst, dstIndex, (long)length); + } } /** - * @see ArrowBuf#setLongLE(int, long). + * Copy data from a given byte array into this ArrowBuf starting at + * a given index. + * @param index starting index (0 based relative to the portion of memory) + * this ArrowBuf has access to + * @param src byte array to copy the data from */ - @Override - public void _setLongLE(int index, long value) { - setLongLE(index, value); - } - - @Override - public ArrowBuf getBytes(int index, ByteBuf dst, int dstIndex, int length) { - udle.getBytes(index + offset, dst, dstIndex, length); - return this; + public void setBytes(int index, byte[] src) { + setBytes(index, src, 0, src.length); } - @Override - public ArrowBuf getBytes(int index, OutputStream out, int length) throws IOException { - udle.getBytes(index + offset, out, length); - return this; - } - - @Override - public int getBytes(int index, GatheringByteChannel out, int length) throws IOException { - return udle.getBytes(index + offset, out, length); + /** + * Copy data from a given byte array starting at the given source index into + * this ArrowBuf at a given index. + * @param index index (0 based relative to the portion of memory this ArrowBuf + * has access to) + * @param src src byte array to copy the data from + * @param srcIndex index in the byte array where the copy will start from + * @param length length of data to copy from byte array + */ + public void setBytes(int index, byte[] src, int srcIndex, int length) { + // bound check for this ArrowBuf where the data will be copied into + checkIndex(index, length); + // null check + Preconditions.checkArgument(src != null, "expecting a valid src byte array"); + // bound check for src byte array where the data will be copied from + if (isOutOfBounds(srcIndex, length, src.length)) { + // not enough space to copy "length" bytes into dst array from dstIndex onwards + throw new IndexOutOfBoundsException("Not enough space to copy data from byte array" + srcIndex); + } + if (length > 0) { + // copy "length" bytes from src byte array at the starting index (srcIndex) + // into this ArrowBuf starting at address "addr(index)" + PlatformDependent.copyMemory(src, srcIndex, addr(index), (long)length); + } } - @Override - public int getBytes(int index, FileChannel out, long position, int length) throws IOException { - return udle.getBytes(index + offset, out, position, length); + /** + * Copy data from this ArrowBuf at a given index into the destination + * ByteBuffer. + * @param index index (0 based relative to the portion of memory this ArrowBuf + * has access to) + * @param dst dst ByteBuffer where the data will be copied into + */ + public void getBytes(int index, ByteBuffer dst) { + // bound check for this ArrowBuf where the data will be copied from + checkIndex(index, dst.remaining()); + // dst.remaining() bytes of data will be copied into dst ByteBuffer + if (dst.remaining() != 0) { + // address in this ArrowBuf where the copy will begin from + final long srcAddress = addr(index); + if (dst.isDirect()) { + if (dst.isReadOnly()) { + throw new ReadOnlyBufferException(); + } + // copy dst.remaining() bytes of data from this ArrowBuf starting + // at address srcAddress into the dst ByteBuffer starting at + // address dstAddress + final long dstAddress = PlatformDependent.directBufferAddress(dst) + (long)dst.position(); + PlatformDependent.copyMemory(srcAddress, dstAddress, (long)dst.remaining()); + // after copy, bump the next write position for the dst ByteBuffer + dst.position(dst.position() + dst.remaining()); + } else if (dst.hasArray()) { + // copy dst.remaining() bytes of data from this ArrowBuf starting + // at address srcAddress into the dst ByteBuffer starting at + // index dstIndex + final int dstIndex = dst.arrayOffset() + dst.position(); + PlatformDependent.copyMemory(srcAddress, dst.array(), dstIndex, (long)dst.remaining()); + // after copy, bump the next write position for the dst ByteBuffer + dst.position(dst.position() + dst.remaining()); + } else { + throw new UnsupportedOperationException("Copy from this ArrowBuf to ByteBuffer is not supported"); + } + } } - @Override - public ArrowBuf setBytes(int index, ByteBuf src, int srcIndex, int length) { - udle.setBytes(index + offset, src, srcIndex, length); - return this; + /** + * Copy data into this ArrowBuf at a given index onwards from + * a source ByteBuffer. + * @param index index index (0 based relative to the portion of memory + * this ArrowBuf has access to) + * @param src src ByteBuffer where the data will be copied from + */ + public void setBytes(int index, ByteBuffer src) { + // bound check for this ArrowBuf where the data will be copied into + checkIndex(index, src.remaining()); + // length of data to copy + int length = src.remaining(); + // address in this ArrowBuf where the data will be copied to + long dstAddress = addr(index); + if (length != 0) { + if (src.isDirect()) { + // copy src.remaining() bytes of data from src ByteBuffer starting at + // address srcAddress into this ArrowBuf starting at address dstAddress + final long srcAddress = PlatformDependent.directBufferAddress(src) + (long)src.position(); + PlatformDependent.copyMemory(srcAddress, dstAddress, (long)length); + // after copy, bump the next read position for the src ByteBuffer + src.position(src.position() + length); + } else if (src.hasArray()) { + // copy src.remaining() bytes of data from src ByteBuffer starting at + // index srcIndex into this ArrowBuf starting at address dstAddress + final int srcIndex = src.arrayOffset() + src.position(); + PlatformDependent.copyMemory(src.array(), srcIndex, dstAddress, (long)length); + // after copy, bump the next read position for the src ByteBuffer + src.position(src.position() + length); + } else { + // copy word at a time + while (length >= LONG_SIZE) { + PlatformDependent.putLong(dstAddress, src.getLong()); + length -= LONG_SIZE; + dstAddress += LONG_SIZE; + } + // copy last byte + while (length > 0) { + PlatformDependent.putByte(dstAddress, src.get()); + --length; + ++dstAddress; + } + } + } } /** - * Copies length bytes from src starting at srcIndex - * to this buffer starting at index. + * Copy data into this ArrowBuf at a given index onwards from + * a source ByteBuffer starting at a given srcIndex for a certain + * length. + * @param index index (0 based relative to the portion of memory + * this ArrowBuf has access to) + * @param src src ByteBuffer where the data will be copied from + * @param srcIndex starting index in the src ByteBuffer where the data copy + * will start from + * @param length length of data to copy from src ByteBuffer */ - public ArrowBuf setBytes(int index, ByteBuffer src, int srcIndex, int length) { + public void setBytes(int index, ByteBuffer src, int srcIndex, int length) { + // bound check for this ArrowBuf where the data will be copied into + checkIndex(index, length); if (src.isDirect()) { - checkIndex(index, length); - PlatformDependent.copyMemory(PlatformDependent.directBufferAddress(src) + srcIndex, this - .memoryAddress() + index, - length); + // copy length bytes of data from src ByteBuffer starting at address + // srcAddress into this ArrowBuf at address dstAddres + final long srcAddress = PlatformDependent.directBufferAddress(src) + (long)srcIndex; + final long dstAddress = addr(index); + PlatformDependent.copyMemory(srcAddress, dstAddress, (long)length); } else { if (srcIndex == 0 && src.capacity() == length) { - udle.setBytes(index + offset, src); + // copy the entire ByteBuffer from start to end of length + setBytes(index, src); } else { ByteBuffer newBuf = src.duplicate(); newBuf.position(srcIndex); newBuf.limit(srcIndex + length); - udle.setBytes(index + offset, newBuf); + setBytes(index, newBuf); } } - - return this; - } - - @Override - public ArrowBuf setBytes(int index, byte[] src, int srcIndex, int length) { - udle.setBytes(index + offset, src, srcIndex, length); - return this; } - @Override - public ArrowBuf setBytes(int index, ByteBuffer src) { - udle.setBytes(index + offset, src); - return this; + /** + * Copy a given length of data from this ArrowBuf starting at a given index + * into a dst ArrowBuf at dstIndex. + * @param index index (0 based relative to the portion of memory + * this ArrowBuf has access to) + * @param dst dst ArrowBuf where the data will be copied into + * @param dstIndex index (0 based relative to the portion of memory + * dst ArrowBuf has access to) + * @param length length of data to copy + */ + public void getBytes(int index, ArrowBuf dst, int dstIndex, int length) { + // bound check for this ArrowBuf where the data will be copied from + checkIndex(index, length); + // bound check for this ArrowBuf where the data will be copied into + Preconditions.checkArgument(dst != null, "expecting a valid ArrowBuf"); + // bound check for dst ArrowBuf + if (isOutOfBounds(dstIndex, length, dst.capacity())) { + throw new IndexOutOfBoundsException(String.format("index: %d, length: %d (expected: range(0, %d))", + dstIndex, length, dst.capacity())); + } + if (length != 0) { + // copy length bytes of data from this ArrowBuf starting at + // address srcAddress into dst ArrowBuf starting at address + // dstAddress + final long srcAddress = addr(index); + final long dstAddress = dst.memoryAddress() + (long)dstIndex; + PlatformDependent.copyMemory(srcAddress, dstAddress, (long)length); + } } - @Override - public int setBytes(int index, InputStream in, int length) throws IOException { - return udle.setBytes(index + offset, in, length); + /** + * Copy data from src ArrowBuf starting at index srcIndex into this + * ArrowBuf at given index. + * @param index index index (0 based relative to the portion of memory + * this ArrowBuf has access to) + * @param src src ArrowBuf where the data will be copied from + * @param srcIndex starting index in the src ArrowBuf where the copy + * will begin from + * @param length length of data to copy from src ArrowBuf + */ + public void setBytes(int index, ArrowBuf src, int srcIndex, int length) { + // bound check for this ArrowBuf where the data will be copied into + checkIndex(index, length); + // null check + Preconditions.checkArgument(src != null, "expecting a valid ArrowBuf"); + // bound check for src ArrowBuf + if (isOutOfBounds(srcIndex, length, src.capacity())) { + throw new IndexOutOfBoundsException(String.format("index: %d, length: %d (expected: range(0, %d))", + index, length, src.capacity())); + } + if (length != 0) { + // copy length bytes of data from src ArrowBuf starting at + // address srcAddress into this ArrowBuf starting at address + // dstAddress + final long srcAddress = src.memoryAddress() + (long)srcIndex; + final long dstAddress = addr(index); + PlatformDependent.copyMemory(srcAddress, dstAddress, (long)length); + } } - @Override - public int setBytes(int index, ScatteringByteChannel in, int length) throws IOException { - return udle.setBytes(index + offset, in, length); + /** + * Copy readableBytes() number of bytes from src ArrowBuf + * starting from its readerIndex into this ArrowBuf starting + * at the given index. + * @param index index index (0 based relative to the portion of memory + * this ArrowBuf has access to) + * @param src src ArrowBuf where the data will be copied from + */ + public void setBytes(int index, ArrowBuf src) { + // null check + Preconditions.checkArgument(src != null, "expecting valid ArrowBuf"); + final int length = src.readableBytes(); + // bound check for this ArrowBuf where the data will be copied into + checkIndex(index, length); + final long srcAddress = src.memoryAddress() + (long)src.readerIndex; + final long dstAddress = addr(index); + PlatformDependent.copyMemory(srcAddress, dstAddress, (long)length); + src.readerIndex(src.readerIndex + length); } - @Override - public int setBytes(int index, FileChannel in, long position, int length) throws IOException { - return udle.setBytes(index + offset, in, position, length); + /** + * Copy a certain length of bytes from given InputStream + * into this ArrowBuf at the provided index. + * @param index index index (0 based relative to the portion of memory + * this ArrowBuf has access to) + * @param in src stream to copy from + * @param length length of data to copy + * @return number of bytes copied from stream into ArrowBuf + * @throws IOException on failing to read from stream + */ + public int setBytes(int index, InputStream in, int length) throws IOException { + Preconditions.checkArgument(in != null, "expecting valid input stream"); + checkIndex(index, length); + int readBytes = 0; + if (length > 0) { + byte[] tmp = new byte[length]; + // read the data from input stream into tmp byte array + readBytes = in.read(tmp); + if (readBytes > 0) { + // copy readBytes length of data from the tmp byte array starting + // at srcIndex 0 into this ArrowBuf starting at address addr(index) + PlatformDependent.copyMemory(tmp, 0, addr(index), readBytes); + } + } + return readBytes; } - @Override - public byte getByte(int index) { - chk(index, 1); - return PlatformDependent.getByte(addr(index)); + /** + * Copy a certain length of bytes from this ArrowBuf at a given + * index into the given OutputStream. + * @param index index index (0 based relative to the portion of memory + * this ArrowBuf has access to) + * @param out dst stream to copy data into + * @param length length of data to copy + * @throws IOException on failing to write to stream + */ + public void getBytes(int index, OutputStream out, int length) throws IOException { + Preconditions.checkArgument(out != null, "expecting valid output stream"); + checkIndex(index, length); + if (length > 0) { + // copy length bytes of data from this ArrowBuf starting at + // address addr(index) into the tmp byte array starting at index 0 + byte[] tmp = new byte[length]; + PlatformDependent.copyMemory(addr(index), tmp, 0, length); + // write the copied data to output stream + out.write(tmp); + } } @Override public void close() { - release(); + referenceManager.release(); } /** - * Returns the possible memory consumed by this ArrowBuf in the worse case scenario. (not - * shared, connected to larger - * underlying buffer of allocated memory) - * + * Returns the possible memory consumed by this ArrowBuf in the worse case scenario. + * (not shared, connected to larger underlying buffer of allocated memory) * @return Size in bytes. */ public int getPossibleMemoryConsumed() { - if (isEmpty) { - return 0; - } - return ledger.getSize(); + return isEmpty ? 0 : referenceManager.getSize(); } /** * Return that is Accounted for by this buffer (and its potentially shared siblings within the - * context of the - * associated allocator). - * + * context of the associated allocator). * @return Size in bytes. */ public int getActualMemoryConsumed() { - if (isEmpty) { - return 0; - } - return ledger.getAccountedSize(); + return isEmpty ? 0 : referenceManager.getAccountedSize(); } /** @@ -1076,24 +1070,12 @@ public String toHexString(final int start, final int length) { /** * Get the integer id assigned to this ArrowBuf for debugging purposes. - * * @return integer id */ public long getId() { return id; } - /** Returns all ledger information with stack traces as a string. */ - public String toVerboseString() { - if (isEmpty) { - return toString(); - } - - StringBuilder sb = new StringBuilder(); - ledger.print(sb, 0, Verbosity.LOG_WITH_STACKTRACE); - return sb.toString(); - } - /** * Prints information of this buffer into sb at the given * indentation and verbosity level. @@ -1111,39 +1093,95 @@ public void print(StringBuilder sb, int indent, Verbosity verbosity) { } } - @Override + /** + * Get the index at which the next byte will be read from. + * @return reader index + */ + public int readerIndex() { + return readerIndex; + } + + /** + * Get the index at which next byte will be written to. + * @return writer index + */ + public int writerIndex() { + return writerIndex; + } + + /** + * Set the reader index for this ArrowBuf. + * @param readerIndex new reader index + * @return this ArrowBuf + */ public ArrowBuf readerIndex(int readerIndex) { - super.readerIndex(readerIndex); + this.readerIndex = readerIndex; return this; } - @Override + /** + * Set the writer index for this ArrowBuf. + * @param writerIndex new writer index + * @return this ArrowBuf + */ public ArrowBuf writerIndex(int writerIndex) { - super.writerIndex(writerIndex); + this.writerIndex = writerIndex; return this; } /** - * The outcome of a Transfer. + * Zero-out the bytes in this ArrowBuf starting at + * the given index for the given length. + * @param index index index (0 based relative to the portion of memory + * this ArrowBuf has access to) + * @param length length of bytes to zero-out + * @return this ArrowBuf */ - public class TransferResult { - - /** - * Whether this transfer fit within the target allocator's capacity. - */ - public final boolean allocationFit; - - /** - * The newly created buffer associated with the target allocator. - */ - public final ArrowBuf buffer; - - private TransferResult(boolean allocationFit, ArrowBuf buffer) { - this.allocationFit = allocationFit; - this.buffer = buffer; + public ArrowBuf setZero(int index, int length) { + if (length == 0) { + return this; + } else { + this.checkIndex(index, length); + int nLong = length >>> 3; + int nBytes = length & 7; + int i; + for (i = nLong; i > 0; --i) { + setLong(index, 0L); + index += 8; + } + if (nBytes == 4) { + setInt(index, 0); + } else if (nBytes < 4) { + for (i = nBytes; i > 0; --i) { + setByte(index, 0); + ++index; + } + } else { + setInt(index, 0); + index += 4; + for (i = nBytes - 4; i > 0; --i) { + setByte(index, 0); + ++index; + } + } + return this; } - } - + /** + * Returns this if size is less then {@link #capacity()}, otherwise + * delegates to {@link BufferManager#replace(ArrowBuf, int)} to get a new buffer. + */ + public ArrowBuf reallocIfNeeded(final int size) { + Preconditions.checkArgument(size >= 0, "reallocation size must be non-negative"); + if (this.capacity() >= size) { + return this; + } + if (bufferManager != null) { + return bufferManager.replace(this, size); + } else { + throw new UnsupportedOperationException( + "Realloc is only available in the context of operator's UDFs"); + } + } } diff --git a/java/memory/src/main/java/io/netty/buffer/ExpandableByteBuf.java b/java/memory/src/main/java/io/netty/buffer/ExpandableByteBuf.java index 62d020cf47a..c2f3f68b0c0 100644 --- a/java/memory/src/main/java/io/netty/buffer/ExpandableByteBuf.java +++ b/java/memory/src/main/java/io/netty/buffer/ExpandableByteBuf.java @@ -41,7 +41,7 @@ public ByteBuf copy(int index, int length) { @Override public ByteBuf capacity(int newCapacity) { if (newCapacity > capacity()) { - ByteBuf newBuf = allocator.buffer(newCapacity); + ByteBuf newBuf = allocator.buffer(newCapacity).asNettyBuffer(); newBuf.writeBytes(buffer, 0, buffer.capacity()); newBuf.readerIndex(buffer.readerIndex()); newBuf.writerIndex(buffer.writerIndex()); diff --git a/java/memory/src/main/java/io/netty/buffer/NettyArrowBuf.java b/java/memory/src/main/java/io/netty/buffer/NettyArrowBuf.java new file mode 100644 index 00000000000..2b831a671cf --- /dev/null +++ b/java/memory/src/main/java/io/netty/buffer/NettyArrowBuf.java @@ -0,0 +1,560 @@ +/* + * 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 io.netty.buffer; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.FileChannel; +import java.nio.channels.GatheringByteChannel; +import java.nio.channels.ScatteringByteChannel; + +import org.apache.arrow.memory.ArrowByteBufAllocator; +import org.apache.arrow.memory.BoundsChecking; +import org.apache.arrow.util.Preconditions; + +import io.netty.util.internal.PlatformDependent; + +/** + * Netty specific wrapper over ArrowBuf for use in Netty framework. + */ +public class NettyArrowBuf extends AbstractByteBuf implements AutoCloseable { + + private final ArrowBuf arrowBuf; + private final ArrowByteBufAllocator arrowByteBufAllocator; + private int length; + private final long address; + + public NettyArrowBuf( + final ArrowBuf arrowBuf, + final ByteBufAllocator arrowByteBufAllocator, + final int length) { + super(length); + this.arrowBuf = arrowBuf; + this.arrowByteBufAllocator = (ArrowByteBufAllocator)arrowByteBufAllocator; + this.length = length; + this.address = arrowBuf.memoryAddress(); + } + + @Override + public ByteBuf copy() { + throw new UnsupportedOperationException(); + } + + @Override + public ByteBuf copy(int index, int length) { + throw new UnsupportedOperationException(); + } + + @Override + public ByteBuf retain() { + arrowBuf.getReferenceManager().retain(); + return this; + } + + @Override + public ByteBuf retain(final int increment) { + arrowBuf.getReferenceManager().retain(increment); + return this; + } + + @Override + public boolean isDirect() { + return true; + } + + @Override + public synchronized ByteBuf capacity(int newCapacity) { + if (newCapacity == length) { + return this; + } + Preconditions.checkArgument(newCapacity >= 0); + if (newCapacity < length) { + length = newCapacity; + return this; + } + throw new UnsupportedOperationException("Buffers don't support resizing that increases the size."); + } + + @Override + public ByteBuf unwrap() { + throw new UnsupportedOperationException("Operation not supported"); + } + + @Override + public int refCnt() { + return arrowBuf.getReferenceManager().getRefCount(); + } + + @Override + public ArrowByteBufAllocator alloc() { + return arrowByteBufAllocator; + } + + @Override + public boolean hasArray() { + return false; + } + + @Override + public byte[] array() { + throw new UnsupportedOperationException("Operation not supported on direct buffer"); + } + + @Override + public int arrayOffset() { + throw new UnsupportedOperationException("Operation not supported on direct buffer"); + } + + @Override + public boolean hasMemoryAddress() { + return true; + } + + @Override + public long memoryAddress() { + return this.address; + } + + @Override + public ByteBuf touch() { + return this; + } + + @Override + public ByteBuf touch(Object hint) { + return this; + } + + @Override + public int capacity() { + return arrowBuf.capacity(); + } + + @Override + public NettyArrowBuf slice() { + return arrowBuf.slice().asNettyBuffer(); + } + + @Override + public NettyArrowBuf slice(int index, int length) { + return arrowBuf.slice(index, length).asNettyBuffer(); + } + + @Override + public void close() { + arrowBuf.close(); + } + + @Override + public boolean release() { + return arrowBuf.getReferenceManager().release(); + } + + @Override + public boolean release(int decrement) { + return arrowBuf.getReferenceManager().release(decrement); + } + + @Override + public NettyArrowBuf readerIndex(int readerIndex) { + super.readerIndex(readerIndex); + return this; + } + + @Override + public NettyArrowBuf writerIndex(int writerIndex) { + super.writerIndex(writerIndex); + return this; + } + + @Override + public int nioBufferCount() { + return 1; + } + + @Override + public ByteBuffer internalNioBuffer(int index, int length) { + throw new UnsupportedOperationException("operation not supported"); + } + + @Override + public ByteBuffer[] nioBuffers() { + return new ByteBuffer[] {nioBuffer()}; + } + + @Override + public ByteBuffer[] nioBuffers(int index, int length) { + return new ByteBuffer[] {nioBuffer(index, length)}; + } + + @Override + public ByteBuffer nioBuffer() { + return nioBuffer(readerIndex(), readableBytes()); + } + + @Override + public ByteBuffer nioBuffer(int index, int length) { + chk(index, length); + final ByteBuffer buffer = getDirectBuffer(); + buffer.position(index).limit(index + length); + return buffer; + } + + /** + * Get this ArrowBuf as a direct {@link ByteBuffer}. + * @return ByteBuffer + */ + private ByteBuffer getDirectBuffer() { + return PlatformDependent.directBuffer(address, length); + } + + @Override + public ByteBuf getBytes(int index, ByteBuffer dst) { + arrowBuf.getBytes(index, dst); + return this; + } + + @Override + public ByteBuf setBytes(int index, ByteBuffer src) { + arrowBuf.setBytes(index, src); + return this; + } + + @Override + public ByteBuf getBytes(int index, byte[] dst, int dstIndex, int length) { + arrowBuf.getBytes(index, dst, dstIndex, length); + return this; + } + + @Override + public ByteBuf setBytes(int index, byte[] src, int srcIndex, int length) { + arrowBuf.setBytes(index, src, srcIndex, length); + return this; + } + + /** + * Determine if the requested {@code index} and {@code length} will fit within {@code capacity}. + * @param index The starting index. + * @param length The length which will be utilized (starting from {@code index}). + * @param capacity The capacity that {@code index + length} is allowed to be within. + * @return {@code true} if the requested {@code index} and {@code length} will fit within {@code capacity}. + * {@code false} if this would result in an index out of bounds exception. + */ + private static boolean isOutOfBounds(int index, int length, int capacity) { + return (index | length | (index + length) | (capacity - (index + length))) < 0; + } + + @Override + public ByteBuf getBytes(int index, ByteBuf dst, int dstIndex, int length) { + chk(index, length); + Preconditions.checkArgument(dst != null, "Expecting valid dst ByteBuffer"); + if (isOutOfBounds(dstIndex, length, dst.capacity())) { + throw new IndexOutOfBoundsException("dstIndex: " + dstIndex + " length: " + length); + } else { + final long srcAddress = addr(index); + if (dst.hasMemoryAddress()) { + final long dstAddress = dst.memoryAddress() + (long)dstIndex; + PlatformDependent.copyMemory(srcAddress, dstAddress, (long)length); + } else if (dst.hasArray()) { + dstIndex += dst.arrayOffset(); + PlatformDependent.copyMemory(srcAddress, dst.array(), dstIndex, (long)length); + } else { + throw new UnsupportedOperationException("Copy to this ByteBuf is not supported"); + } + } + return this; + } + + @Override + public ByteBuf setBytes(int index, ByteBuf src, int srcIndex, int length) { + chk(index, length); + Preconditions.checkArgument(src != null, "Expecting valid src ByteBuffer"); + if (isOutOfBounds(srcIndex, length, src.capacity())) { + throw new IndexOutOfBoundsException("srcIndex: " + srcIndex + " length: " + length); + } else { + if (length != 0) { + final long dstAddress = addr(index); + if (src.hasMemoryAddress()) { + final long srcAddress = src.memoryAddress() + (long)srcIndex; + PlatformDependent.copyMemory(srcAddress, dstAddress, (long)length); + } else if (src.hasArray()) { + srcIndex += src.arrayOffset(); + PlatformDependent.copyMemory(src.array(), srcIndex, dstAddress, (long)length); + } else { + throw new UnsupportedOperationException("Copy from this ByteBuf is not supported"); + } + } + } + return this; + } + + @Override + public ByteBuf getBytes(int index, OutputStream out, int length) throws IOException { + arrowBuf.getBytes(index, out, length); + return this; + } + + @Override + public int setBytes(int index, InputStream in, int length) throws IOException { + return arrowBuf.setBytes(index, in, length); + } + + @Override + public int getBytes(int index, GatheringByteChannel out, int length) throws IOException { + Preconditions.checkArgument(out != null, "expecting valid gathering byte channel"); + chk(index, length); + if (length == 0) { + return 0; + } else { + final ByteBuffer tmpBuf = getDirectBuffer(); + tmpBuf.clear().position(index).limit(index + length); + return out.write(tmpBuf); + } + } + + @Override + public int getBytes(int index, FileChannel out, long position, int length) throws IOException { + chk(index, length); + if (length == 0) { + return 0; + } else { + final ByteBuffer tmpBuf = getDirectBuffer(); + tmpBuf.clear().position(index).limit(index + length); + return out.write(tmpBuf, position); + } + } + + @Override + public int setBytes(int index, ScatteringByteChannel in, int length) throws IOException { + throw new UnsupportedOperationException("Operation not supported"); + } + + @Override + public int setBytes(int index, FileChannel in, long position, int length) throws IOException { + throw new UnsupportedOperationException("Operation not supported"); + } + + @Override + public ByteOrder order() { + return ByteOrder.LITTLE_ENDIAN; + } + + @Override + public ByteBuf order(ByteOrder endianness) { + return this; + } + + @Override + protected int _getUnsignedMedium(int index) { + return getUnsignedMedium(index); + } + + @Override + protected int _getUnsignedMediumLE(int index) { + return getUnsignedMediumLE(index); + } + + + /*-------------------------------------------------* + | | + | get() APIs | + | | + *-------------------------------------------------*/ + + + @Override + protected byte _getByte(int index) { + return getByte(index); + } + + @Override + public byte getByte(int index) { + return arrowBuf.getByte(index); + } + + @Override + protected short _getShortLE(int index) { + return getShort(index); + } + + @Override + protected short _getShort(int index) { + return getShort(index); + } + + @Override + public short getShort(int index) { + return arrowBuf.getShort(index); + } + + @Override + protected int _getIntLE(int index) { + return getInt(index); + } + + @Override + protected int _getInt(int index) { + return getInt(index); + } + + @Override + public int getInt(int index) { + return arrowBuf.getInt(index); + } + + @Override + protected long _getLongLE(int index) { + return getLong(index); + } + + @Override + protected long _getLong(int index) { + return getLong(index); + } + + @Override + public long getLong(int index) { + return arrowBuf.getLong(index); + } + + + /*-------------------------------------------------* + | | + | set() APIs | + | | + *-------------------------------------------------*/ + + + @Override + protected void _setByte(int index, int value) { + setByte(index, value); + } + + @Override + public NettyArrowBuf setByte(int index, int value) { + arrowBuf.setByte(index, value); + return this; + } + + @Override + protected void _setShortLE(int index, int value) { + setShort(index, value); + } + + @Override + protected void _setShort(int index, int value) { + setShort(index, value); + } + + @Override + public NettyArrowBuf setShort(int index, int value) { + arrowBuf.setShort(index, value); + return this; + } + + private long addr(int index) { + return address + index; + } + + /** + * Helper function to do bounds checking at a particular + * index for particular length of data. + * @param index index (0 based relative to this ArrowBuf) + * @param fieldLength provided length of data for get/set + */ + private void chk(int index, int fieldLength) { + if (BoundsChecking.BOUNDS_CHECKING_ENABLED) { + // check reference count + ensureAccessible(); + // check bounds + if (fieldLength < 0) { + throw new IllegalArgumentException("length: " + fieldLength + " (expected: >= 0)"); + } + if (index < 0 || index > capacity() - fieldLength) { + throw new IndexOutOfBoundsException(String.format( + "index: %d, length: %d (expected: range(0, %d))", index, fieldLength, capacity())); + } + } + } + + @Override + protected void _setMedium(int index, int value) { + setMedium(index, value); + } + + @Override + protected void _setMediumLE(int index, int value) { + setMedium(index, value); + } + + @Override + public NettyArrowBuf setMedium(int index, int value) { + chk(index, 3); + final long addr = addr(index); + // we need to store 3 bytes starting from least significant byte + // and ignoring the most significant byte + // since arrow memory format is little endian, we will + // first store the first 2 bytes followed by third byte + // example: if the 4 byte int value is ABCD where A is MSB + // D is LSB then we effectively want to store DCB in increasing + // address to get Little Endian byte order + // (short)value will give us CD and PlatformDependent.putShort() + // will store them in LE order as DC starting at address addr + // in order to get B, we do ABCD >>> 16 = 00AB => (byte)AB which + // gives B. We store this at address addr + 2. So finally we get + // DCB + PlatformDependent.putShort(addr, (short) value); + PlatformDependent.putByte(addr + 2, (byte) (value >>> 16)); + return this; + } + + @Override + protected void _setInt(int index, int value) { + setInt(index, value); + } + + @Override + protected void _setIntLE(int index, int value) { + setInt(index, value); + } + + @Override + public NettyArrowBuf setInt(int index, int value) { + arrowBuf.setInt(index, value); + return this; + } + + @Override + protected void _setLong(int index, long value) { + setLong(index, value); + } + + @Override + public void _setLongLE(int index, long value) { + setLong(index, value); + } + + @Override + public NettyArrowBuf setLong(int index, long value) { + arrowBuf.setLong(index, value); + return this; + } +} diff --git a/java/memory/src/main/java/org/apache/arrow/memory/AllocationManager.java b/java/memory/src/main/java/org/apache/arrow/memory/AllocationManager.java index 3a8d4651dd3..993ae5d3766 100644 --- a/java/memory/src/main/java/org/apache/arrow/memory/AllocationManager.java +++ b/java/memory/src/main/java/org/apache/arrow/memory/AllocationManager.java @@ -17,17 +17,10 @@ package org.apache.arrow.memory; -import static org.apache.arrow.memory.BaseAllocator.indent; - -import java.util.IdentityHashMap; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -import org.apache.arrow.memory.BaseAllocator.Verbosity; -import org.apache.arrow.memory.util.HistoricalLog; import org.apache.arrow.util.Preconditions; -import io.netty.buffer.ArrowBuf; import io.netty.buffer.PooledByteBufAllocatorL; import io.netty.buffer.UnsafeDirectLittleEndian; @@ -57,7 +50,6 @@ public class AllocationManager { private static final AtomicLong MANAGER_ID_GENERATOR = new AtomicLong(0); - private static final AtomicLong LEDGER_ID_GENERATOR = new AtomicLong(0); private static final PooledByteBufAllocatorL INNER_ALLOCATOR = new PooledByteBufAllocatorL(); static final UnsafeDirectLittleEndian EMPTY = INNER_ALLOCATOR.empty; @@ -66,12 +58,15 @@ public class AllocationManager { private final RootAllocator root; private final long allocatorManagerId = MANAGER_ID_GENERATOR.incrementAndGet(); private final int size; - private final UnsafeDirectLittleEndian underlying; + private final UnsafeDirectLittleEndian memoryChunk; // ARROW-1627 Trying to minimize memory overhead caused by previously used IdentityHashMap // see JIRA for details private final LowCostIdentityHashMap map = new LowCostIdentityHashMap<>(); private final long amCreationTime = System.nanoTime(); + // The ReferenceManager created at the time of creation of this AllocationManager + // is treated as the owning reference manager for the underlying chunk of memory + // managed by this allocation manager private volatile BufferLedger owningLedger; private volatile long amDestructionTime = 0; @@ -80,22 +75,37 @@ public class AllocationManager { accountingAllocator.assertOpen(); this.root = accountingAllocator.root; - this.underlying = INNER_ALLOCATOR.allocate(size); + this.memoryChunk = INNER_ALLOCATOR.allocate(size); // we do a no retain association since our creator will want to retrieve the newly created - // ledger and will create a - // reference count at that point + // ledger and will create a reference count at that point this.owningLedger = associate(accountingAllocator, false); - this.size = underlying.capacity(); + this.size = memoryChunk.capacity(); + } + + BufferLedger getOwningLedger() { + return owningLedger; + } + + void setOwningLedger(final BufferLedger ledger) { + this.owningLedger = ledger; + } + + /** + * Get the underlying memory chunk managed by this AllocationManager. + * @return buffer + */ + UnsafeDirectLittleEndian getMemoryChunk() { + return memoryChunk; } /** * Associate the existing underlying buffer with a new allocator. This will increase the - * reference count to the - * provided ledger by 1. + * reference count on the corresponding buffer ledger by 1 * * @param allocator The target allocator to associate this buffer with. - * @return The Ledger (new or existing) that associates the underlying buffer to this new ledger. + * @return The reference manager (new or existing) that associates the underlying + * buffer to this new ledger. */ BufferLedger associate(final BaseAllocator allocator) { return associate(allocator, true); @@ -103,338 +113,94 @@ BufferLedger associate(final BaseAllocator allocator) { private BufferLedger associate(final BaseAllocator allocator, final boolean retain) { allocator.assertOpen(); - - if (root != allocator.root) { - throw new IllegalStateException( - "A buffer can only be associated between two allocators that share the same root."); - } + Preconditions.checkState(root == allocator.root, + "A buffer can only be associated between two allocators that share the same root"); synchronized (this) { BufferLedger ledger = map.get(allocator); if (ledger != null) { if (retain) { - ledger.inc(); + // bump the ref count for the ledger + ledger.increment(); } return ledger; } - ledger = new BufferLedger(allocator); + ledger = new BufferLedger(allocator, this); + if (retain) { - ledger.inc(); + // the new reference manager will have a ref count of 1 + ledger.increment(); } + + // store the mapping for BufferLedger oldLedger = map.put(ledger); - Preconditions.checkArgument(oldLedger == null); + Preconditions.checkState(oldLedger == null, + "Detected inconsitent state: A reference manager already exists for this allocator"); + + // needed for debugging only: keep a pointer to reference manager inside allocator + // to dump state, verify allocator state etc allocator.associateLedger(ledger); return ledger; } } /** - * The way that a particular BufferLedger communicates back to the AllocationManager that it - * now longer needs to hold - * a reference to particular piece of memory. - * Can only be called when you already hold the lock. + * The way that a particular ReferenceManager (BufferLedger) communicates back to the + * AllocationManager that it no longer needs to hold a reference to a particular + * piece of memory. Reference manager needs to hold a lock to invoke this method + * It is called when the shared refcount of all the ArrowBufs managed by the + * calling ReferenceManager drops to 0. */ - private void release(final BufferLedger ledger) { - final BaseAllocator allocator = ledger.getAllocator(); + void release(final BufferLedger ledger) { + final BaseAllocator allocator = (BaseAllocator)ledger.getAllocator(); allocator.assertOpen(); + // remove the mapping for the allocator + // of calling BufferLedger + Preconditions.checkState(map.containsKey(allocator), + "Expecting a mapping for allocator and reference manager"); final BufferLedger oldLedger = map.remove(allocator); - oldLedger.allocator.dissociateLedger(oldLedger); + + // needed for debug only: tell the allocator that AllocationManager is removing a + // reference manager associated with this particular allocator + ((BaseAllocator)oldLedger.getAllocator()).dissociateLedger(oldLedger); if (oldLedger == owningLedger) { + // the release call was made by the owning reference manager if (map.isEmpty()) { - // no one else owns, lets release. - oldLedger.allocator.releaseBytes(size); - underlying.release(); - oldLedger.allocator.listener.onRelease(size); + // the only mapping was for the owner + // which now has been removed, it implies we can safely destroy the + // underlying memory chunk as it is no longer being referenced + ((BaseAllocator)oldLedger.getAllocator()).releaseBytes(size); + // free the memory chunk associated with the allocation manager + memoryChunk.release(); + ((BaseAllocator)oldLedger.getAllocator()).getListener().onRelease(size); amDestructionTime = System.nanoTime(); owningLedger = null; } else { - // we need to change the owning allocator. we've been removed so we'll get whatever is - // top of list - BufferLedger newLedger = map.getNextValue(); - - // we'll forcefully transfer the ownership and not worry about whether we exceeded the - // limit - // since this consumer can't do anything with this. - oldLedger.transferBalance(newLedger); + // since the refcount dropped to 0 for the owning reference manager and allocation + // manager will no longer keep a mapping for it, we need to change the owning + // reference manager to whatever the next available + // mapping exists. + BufferLedger newOwningLedger = map.getNextValue(); + // we'll forcefully transfer the ownership and not worry about whether we + // exceeded the limit since this consumer can't do anything with this. + oldLedger.transferBalance(newOwningLedger); } } else { - if (map.isEmpty()) { - throw new IllegalStateException("The final removal of a ledger should be connected to " + - "the owning ledger."); - } + // the release call was made by a non-owning reference manager, so after remove there have + // to be 1 or more mappings + Preconditions.checkState(map.size() > 0, + "The final removal of reference manager should be connected to owning reference manager"); } } /** - * The reference manager that binds an allocator manager to a particular BaseAllocator. Also - * responsible for creating - * a set of ArrowBufs that share a common fate and set of reference counts. - * As with AllocationManager, the only reason this is public is due to ArrowBuf being in io - * .netty.buffer package. + * Return the size of underlying chunk of memory managed by this Allocation Manager. + * @return size of memory chunk */ - public class BufferLedger implements ValueWithKeyIncluded { - - private final IdentityHashMap buffers = - BaseAllocator.DEBUG ? new IdentityHashMap<>() : null; - - private final long ledgerId = LEDGER_ID_GENERATOR.incrementAndGet(); // unique ID assigned to - // each ledger - private final AtomicInteger bufRefCnt = new AtomicInteger(0); // start at zero so we can - // manage request for retain - // correctly - private final long lCreationTime = System.nanoTime(); - private final BaseAllocator allocator; - private final HistoricalLog historicalLog = - BaseAllocator.DEBUG ? new HistoricalLog(BaseAllocator.DEBUG_LOG_LENGTH, "BufferLedger[%d]", 1) : null; - private volatile long lDestructionTime = 0; - - private BufferLedger(BaseAllocator allocator) { - this.allocator = allocator; - } - - /** - * Get the allocator for this ledger. - * @return allocator - */ - private BaseAllocator getAllocator() { - return allocator; - } - - @Override - public BaseAllocator getKey() { - return allocator; - } - - /** - * Transfer any balance the current ledger has to the target ledger. In the case that the - * current ledger holds no - * memory, no transfer is made to the new ledger. - * - * @param target The ledger to transfer ownership account to. - * @return Whether transfer fit within target ledgers limits. - */ - public boolean transferBalance(final BufferLedger target) { - Preconditions.checkNotNull(target); - Preconditions.checkArgument(allocator.root == target.allocator.root, - "You can only transfer between two allocators that share the same root."); - allocator.assertOpen(); - - target.allocator.assertOpen(); - // if we're transferring to ourself, just return. - if (target == this) { - return true; - } - - // since two balance transfers out from the allocator manager could cause incorrect - // accounting, we need to ensure - // that this won't happen by synchronizing on the allocator manager instance. - synchronized (AllocationManager.this) { - if (owningLedger != this) { - return true; - } - - if (BaseAllocator.DEBUG) { - this.historicalLog.recordEvent("transferBalance(%s)", target.allocator.name); - target.historicalLog.recordEvent("incoming(from %s)", owningLedger.allocator.name); - } - - boolean overlimit = target.allocator.forceAllocate(size); - allocator.releaseBytes(size); - owningLedger = target; - return overlimit; - } - - } - - /** - * Print the current ledger state to the provided StringBuilder. - * - * @param sb The StringBuilder to populate. - * @param indent The level of indentation to position the data. - * @param verbosity The level of verbosity to print. - */ - public void print(StringBuilder sb, int indent, Verbosity verbosity) { - indent(sb, indent) - .append("ledger[") - .append(ledgerId) - .append("] allocator: ") - .append(allocator.name) - .append("), isOwning: ") - .append(owningLedger == this) - .append(", size: ") - .append(size) - .append(", references: ") - .append(bufRefCnt.get()) - .append(", life: ") - .append(lCreationTime) - .append("..") - .append(lDestructionTime) - .append(", allocatorManager: [") - .append(AllocationManager.this.allocatorManagerId) - .append(", life: ") - .append(amCreationTime) - .append("..") - .append(amDestructionTime); - - if (!BaseAllocator.DEBUG) { - sb.append("]\n"); - } else { - synchronized (buffers) { - sb.append("] holds ") - .append(buffers.size()) - .append(" buffers. \n"); - for (ArrowBuf buf : buffers.keySet()) { - buf.print(sb, indent + 2, verbosity); - sb.append('\n'); - } - } - } - - } - - private void inc() { - bufRefCnt.incrementAndGet(); - } - - /** - * Decrement the ledger's reference count. If the ledger is decremented to zero, this ledger - * should release its - * ownership back to the AllocationManager - * - * @param decrement amount to decrease the reference count by - * @return the new reference count - */ - public int decrement(int decrement) { - allocator.assertOpen(); - - final int outcome; - synchronized (AllocationManager.this) { - outcome = bufRefCnt.addAndGet(-decrement); - if (outcome == 0) { - lDestructionTime = System.nanoTime(); - release(this); - } - } - - return outcome; - } - - /** - * Returns the ledger associated with a particular BufferAllocator. If the BufferAllocator - * doesn't currently have a - * ledger associated with this AllocationManager, a new one is created. This is placed on - * BufferLedger rather than - * AllocationManager directly because ArrowBufs don't have access to AllocationManager and - * they are the ones - * responsible for exposing the ability to associate multiple allocators with a particular - * piece of underlying - * memory. Note that this will increment the reference count of this ledger by one to ensure - * the ledger isn't - * destroyed before use. - * - * @param allocator A BufferAllocator. - * @return The ledger associated with the BufferAllocator. - */ - public BufferLedger getLedgerForAllocator(BufferAllocator allocator) { - return associate((BaseAllocator) allocator); - } - - /** - * Create a new ArrowBuf associated with this AllocationManager and memory. Does not impact - * reference count. - * Typically used for slicing. - * - * @param offset The offset in bytes to start this new ArrowBuf. - * @param length The length in bytes that this ArrowBuf will provide access to. - * @return A new ArrowBuf that shares references with all ArrowBufs associated with this BufferLedger - */ - public ArrowBuf newArrowBuf(int offset, int length) { - allocator.assertOpen(); - return newArrowBuf(offset, length, null); - } - - /** - * Create a new ArrowBuf associated with this AllocationManager and memory. - * - * @param offset The offset in bytes to start this new ArrowBuf. - * @param length The length in bytes that this ArrowBuf will provide access to. - * @param manager An optional BufferManager argument that can be used to manage expansion of - * this ArrowBuf - * @return A new ArrowBuf that shares references with all ArrowBufs associated with this BufferLedger - */ - public ArrowBuf newArrowBuf(int offset, int length, BufferManager manager) { - allocator.assertOpen(); - - final ArrowBuf buf = new ArrowBuf( - bufRefCnt, - this, - underlying, - manager, - allocator.getAsByteBufAllocator(), - offset, - length, - false); - - if (BaseAllocator.DEBUG) { - historicalLog.recordEvent( - "ArrowBuf(BufferLedger, BufferAllocator[%s], " + - "UnsafeDirectLittleEndian[identityHashCode == " + - "%d](%s)) => ledger hc == %d", - allocator.name, System.identityHashCode(buf), buf.toString(), - System.identityHashCode(this)); - - synchronized (buffers) { - buffers.put(buf, null); - } - } - - return buf; - - } - - /** - * What is the total size (in bytes) of memory underlying this ledger. - * - * @return Size in bytes - */ - public int getSize() { - return size; - } - - /** - * How much memory is accounted for by this ledger. This is either getSize() if this is the - * owning ledger for the - * memory or zero in the case that this is not the owning ledger associated with this memory. - * - * @return Amount of accounted(owned) memory associated with this ledger. - */ - public int getAccountedSize() { - synchronized (AllocationManager.this) { - if (owningLedger == this) { - return size; - } else { - return 0; - } - } - } - - /** - * Package visible for debugging/verification only. - */ - UnsafeDirectLittleEndian getUnderlying() { - return underlying; - } - - /** - * Package visible for debugging/verification only. - */ - boolean isOwningLedger() { - return this == owningLedger; - } - + public int getSize() { + return size; } - } diff --git a/java/memory/src/main/java/org/apache/arrow/memory/ArrowByteBufAllocator.java b/java/memory/src/main/java/org/apache/arrow/memory/ArrowByteBufAllocator.java index 8c0a3150a67..b6296cfc723 100644 --- a/java/memory/src/main/java/org/apache/arrow/memory/ArrowByteBufAllocator.java +++ b/java/memory/src/main/java/org/apache/arrow/memory/ArrowByteBufAllocator.java @@ -53,7 +53,7 @@ public ByteBuf buffer() { @Override public ByteBuf buffer(int initialCapacity) { - return new ExpandableByteBuf(allocator.buffer(initialCapacity), allocator); + return new ExpandableByteBuf(allocator.buffer(initialCapacity).asNettyBuffer(), allocator); } @Override @@ -83,7 +83,7 @@ public ByteBuf directBuffer() { @Override public ByteBuf directBuffer(int initialCapacity) { - return allocator.buffer(initialCapacity); + return allocator.buffer(initialCapacity).asNettyBuffer(); } @Override diff --git a/java/memory/src/main/java/org/apache/arrow/memory/BaseAllocator.java b/java/memory/src/main/java/org/apache/arrow/memory/BaseAllocator.java index bd84aefe190..b11678e8bb9 100644 --- a/java/memory/src/main/java/org/apache/arrow/memory/BaseAllocator.java +++ b/java/memory/src/main/java/org/apache/arrow/memory/BaseAllocator.java @@ -22,7 +22,6 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.arrow.memory.AllocationManager.BufferLedger; import org.apache.arrow.memory.util.AssertionUtil; import org.apache.arrow.memory.util.HistoricalLog; import org.apache.arrow.util.Preconditions; @@ -43,8 +42,7 @@ public abstract class BaseAllocator extends Accountant implements BufferAllocato public static final int DEBUG_LOG_LENGTH = 6; public static final boolean DEBUG = AssertionUtil.isAssertionsEnabled() || Boolean.parseBoolean(System.getProperty(DEBUG_ALLOCATOR, "false")); - private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseAllocator - .class); + private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseAllocator.class); // Package exposed for sharing between AllocatorManger and BaseAllocator objects final String name; final RootAllocator root; @@ -71,11 +69,11 @@ public abstract class BaseAllocator extends Accountant implements BufferAllocato * construction */ protected BaseAllocator( - final BaseAllocator parentAllocator, - final AllocationListener listener, - final String name, - final long initReservation, - final long maxAllocation) throws OutOfMemoryException { + final BaseAllocator parentAllocator, + final AllocationListener listener, + final String name, + final long initReservation, + final long maxAllocation) throws OutOfMemoryException { super(parentAllocator, initReservation, maxAllocation); this.listener = listener; @@ -88,7 +86,7 @@ protected BaseAllocator( empty = createEmpty(); } else { throw new IllegalStateException("An parent allocator must either carry a root or be the " + - "root."); + "root."); } this.parentAllocator = parentAllocator; @@ -111,6 +109,10 @@ protected BaseAllocator( } + AllocationListener getListener() { + return listener; + } + private static String createErrorMsg(final BufferAllocator allocator, final int rounded, final int requested) { if (rounded != requested) { return String.format( @@ -119,7 +121,7 @@ private static String createErrorMsg(final BufferAllocator allocator, final int } else { return String.format( "Unable to allocate buffer of size %d due to memory limit. Current " + - "allocation: %d", rounded, allocator.getAllocatedMemory()); + "allocation: %d", rounded, allocator.getAllocatedMemory()); } } @@ -259,7 +261,7 @@ public ArrowBuf buffer(final int initialRequestSize) { } private ArrowBuf createEmpty() { - return new ArrowBuf(new AtomicInteger(), null, AllocationManager.EMPTY, null, null, 0, 0, true); + return new ArrowBuf(ReferenceManager.NO_OP, null, 0, 0, true); } @Override @@ -274,9 +276,9 @@ public ArrowBuf buffer(final int initialRequestSize, BufferManager manager) { // round to next largest power of two if we're within a chunk since that is how our allocator // operates - final int actualRequestSize = initialRequestSize < AllocationManager.CHUNK_SIZE ? - nextPowerOfTwo(initialRequestSize) - : initialRequestSize; + final int actualRequestSize = + initialRequestSize < AllocationManager.CHUNK_SIZE ? + nextPowerOfTwo(initialRequestSize) : initialRequestSize; listener.onPreAllocation(actualRequestSize); @@ -317,17 +319,17 @@ public ArrowBuf buffer(final int initialRequestSize, BufferManager manager) { } /** - * Used by usual allocation as well as for allocating a pre-reserved buffer. Skips the typical - * accounting associated - * with creating a new buffer. + * Used by usual allocation as well as for allocating a pre-reserved buffer. + * Skips the typical accounting associated with creating a new buffer. */ - private ArrowBuf bufferWithoutReservation(final int size, BufferManager bufferManager) throws - OutOfMemoryException { + private ArrowBuf bufferWithoutReservation( + final int size, + BufferManager bufferManager) throws OutOfMemoryException { assertOpen(); final AllocationManager manager = new AllocationManager(this, size); final BufferLedger ledger = manager.associate(this); // +1 ref cnt (required) - final ArrowBuf buffer = ledger.newArrowBuf(0, size, bufferManager); + final ArrowBuf buffer = ledger.newArrowBuf(size, bufferManager); // make sure that our allocation is equal to what we expected. Preconditions.checkArgument(buffer.capacity() == size, @@ -357,8 +359,8 @@ public BufferAllocator newChildAllocator( final long maxAllocation) { assertOpen(); - final ChildAllocator childAllocator = new ChildAllocator(listener, this, name, initReservation, - maxAllocation); + final ChildAllocator childAllocator = + new ChildAllocator(listener, this, name, initReservation, maxAllocation); if (DEBUG) { synchronized (DEBUG_LOCK) { @@ -406,23 +408,23 @@ public synchronized void close() { } throw new IllegalStateException( - String.format("Allocator[%s] closed with outstanding child allocators.\n%s", name, - toString())); + String.format("Allocator[%s] closed with outstanding child allocators.\n%s", name, + toString())); } // are there outstanding buffers? final int allocatedCount = childLedgers.size(); if (allocatedCount > 0) { throw new IllegalStateException( - String.format("Allocator[%s] closed with outstanding buffers allocated (%d).\n%s", - name, allocatedCount, toString())); + String.format("Allocator[%s] closed with outstanding buffers allocated (%d).\n%s", + name, allocatedCount, toString())); } if (reservations.size() != 0) { throw new IllegalStateException( - String.format("Allocator[%s] closed with outstanding reservations (%d).\n%s", name, - reservations.size(), - toString())); + String.format("Allocator[%s] closed with outstanding reservations (%d).\n%s", name, + reservations.size(), + toString())); } } @@ -432,8 +434,8 @@ public synchronized void close() { final long allocated = getAllocatedMemory(); if (allocated > 0) { throw new IllegalStateException( - String.format("Memory was leaked by query. Memory leaked: (%d)\n%s", allocated, - toString())); + String.format("Memory was leaked by query. Memory leaked: (%d)\n%s", allocated, + toString())); } // we need to release our memory to our parent before we tell it we've closed. @@ -446,9 +448,7 @@ public synchronized void close() { if (DEBUG) { historicalLog.recordEvent("closed"); - logger.debug(String.format( - "closed allocator[%s].", - name)); + logger.debug(String.format("closed allocator[%s].", name)); } @@ -487,24 +487,21 @@ private void hist(String noteFormat, Object... args) { * @throws IllegalStateException when any problems are found */ void verifyAllocator() { - final IdentityHashMap buffersSeen = new - IdentityHashMap<>(); - verifyAllocator(buffersSeen); + final IdentityHashMap seen = new IdentityHashMap<>(); + verifyAllocator(seen); } /** - * Verifies the accounting state of the allocator. Only works for DEBUG. - * - *

This overload is used for recursive calls, allowing for checking that ArrowBufs are unique - * across all allocators - * that are checked. + * Verifies the accounting state of the allocator (Only works for DEBUG) + * This overload is used for recursive calls, allowing for checking + * that ArrowBufs are unique across all allocators that are checked. * * @param buffersSeen a map of buffers that have already been seen when walking a tree of * allocators * @throws IllegalStateException when any problems are found */ - private void verifyAllocator(final IdentityHashMap - buffersSeen) { + private void verifyAllocator( + final IdentityHashMap buffersSeen) { // The remaining tests can only be performed if we're in debug mode. if (!DEBUG) { return; @@ -538,8 +535,8 @@ private void verifyAllocator(final IdentityHashMap, ReferenceManager { + private final IdentityHashMap buffers = + BaseAllocator.DEBUG ? new IdentityHashMap<>() : null; + private static final AtomicLong LEDGER_ID_GENERATOR = new AtomicLong(0); + // unique ID assigned to each ledger + private final long ledgerId = LEDGER_ID_GENERATOR.incrementAndGet(); + private final AtomicInteger bufRefCnt = new AtomicInteger(0); // start at zero so we can + // manage request for retain + // correctly + private final long lCreationTime = System.nanoTime(); + private final BaseAllocator allocator; + private final AllocationManager allocationManager; + private final HistoricalLog historicalLog = + BaseAllocator.DEBUG ? new HistoricalLog(BaseAllocator.DEBUG_LOG_LENGTH, + "BufferLedger[%d]", 1) : null; + private volatile long lDestructionTime = 0; + + BufferLedger(final BaseAllocator allocator, final AllocationManager allocationManager) { + this.allocator = allocator; + this.allocationManager = allocationManager; + } + + boolean isOwningLedger() { + return this == allocationManager.getOwningLedger(); + } + + public BaseAllocator getKey() { + return allocator; + } + + /** + * Get the buffer allocator associated with this reference manager. + * @return buffer allocator + */ + @Override + public BufferAllocator getAllocator() { + return allocator; + } + + /** + * Get this ledger's reference count. + * @return reference count + */ + @Override + public int getRefCount() { + return bufRefCnt.get(); + } + + /** + * Increment the ledger's reference count for the associated + * underlying memory chunk. All ArrowBufs managed by this ledger + * will share the ref count. + */ + void increment() { + bufRefCnt.incrementAndGet(); + } + + /** + * Decrement the ledger's reference count by 1 for the associated underlying + * memory chunk. If the reference count drops to 0, it implies that + * no ArrowBufs managed by this reference manager need access to the memory + * chunk. In that case, the ledger should inform the allocation manager + * about releasing its ownership for the chunk. Whether or not the memory + * chunk will be released is something that {@link AllocationManager} will + * decide since tracks the usage of memory chunk across multiple reference + * managers and allocators. + * @return true if the new ref count has dropped to 0, false otherwise + */ + @Override + public boolean release() { + return release(1); + } + + /** + * Decrement the ledger's reference count for the associated underlying + * memory chunk. If the reference count drops to 0, it implies that + * no ArrowBufs managed by this reference manager need access to the memory + * chunk. In that case, the ledger should inform the allocation manager + * about releasing its ownership for the chunk. Whether or not the memory + * chunk will be released is something that {@link AllocationManager} will + * decide since tracks the usage of memory chunk across multiple reference + * managers and allocators. + * @param decrement amount to decrease the reference count by + * @return true if the new ref count has dropped to 0, false otherwise + */ + @Override + public boolean release(int decrement) { + Preconditions.checkState(decrement >= 1, + "ref count decrement should be greater than or equal to 1"); + // decrement the ref count + final int refCnt = decrement(decrement); + if (BaseAllocator.DEBUG) { + historicalLog.recordEvent("release(%d). original value: %d", + decrement, refCnt + decrement); + } + // the new ref count should be >= 0 + Preconditions.checkState(refCnt >= 0, "RefCnt has gone negative"); + return refCnt == 0; + } + + /** + * Decrement the ledger's reference count for the associated underlying + * memory chunk. If the reference count drops to 0, it implies that + * no ArrowBufs managed by this reference manager need access to the memory + * chunk. In that case, the ledger should inform the allocation manager + * about releasing its ownership for the chunk. Whether or not the memory + * chunk will be released is something that {@link AllocationManager} will + * decide since tracks the usage of memory chunk across multiple reference + * managers and allocators. + * + * @param decrement amount to decrease the reference count by + * @return the new reference count + */ + private int decrement(int decrement) { + allocator.assertOpen(); + final int outcome; + synchronized (allocationManager) { + outcome = bufRefCnt.addAndGet(-decrement); + if (outcome == 0) { + lDestructionTime = System.nanoTime(); + // refcount of this reference manager has dropped to 0 + // inform the allocation manager that this reference manager + // no longer holds references to underlying memory + allocationManager.release(this); + } + } + return outcome; + } + + /** + * Increment the ledger's reference count for associated + * underlying memory chunk by 1. + */ + @Override + public void retain() { + retain(1); + } + + /** + * Increment the ledger's reference count for associated + * underlying memory chunk by the given amount. + * + * @param increment amount to increase the reference count by + */ + @Override + public void retain(int increment) { + Preconditions.checkArgument(increment > 0, "retain(%d) argument is not positive", increment); + if (BaseAllocator.DEBUG) { + historicalLog.recordEvent("retain(%d)", increment); + } + final int originalReferenceCount = bufRefCnt.getAndAdd(increment); + Preconditions.checkArgument(originalReferenceCount > 0); + } + + /** + * Derive a new ArrowBuf from a given source ArrowBuf. The new derived + * ArrowBuf will share the same reference count as rest of the ArrowBufs + * associated with this ledger. This operation is typically used for + * slicing -- creating new ArrowBufs from a compound ArrowBuf starting at + * a particular index in the underlying memory and having access to a + * particular length (in bytes) of data in memory chunk. + *

+ * This method is also used as a helper for transferring ownership and retain to target + * allocator. + *

+ * @param sourceBuffer source ArrowBuf + * @param index index (relative to source ArrowBuf) new ArrowBuf should be + * derived from + * @param length length (bytes) of data in underlying memory that derived buffer will + * have access to in underlying memory + * @return derived buffer + */ + @Override + public ArrowBuf deriveBuffer(final ArrowBuf sourceBuffer, int index, int length) { + /* + * Usage type 1 for deriveBuffer(): + * Used for slicing where index represents a relative index in the source ArrowBuf + * as the slice start point. This is why we need to add the source buffer offset + * to compute the start virtual address of derived buffer within the + * underlying chunk. + * + * Usage type 2 for deriveBuffer(): + * Used for retain(target allocator) and transferOwnership(target allocator) + * where index is 0 since these operations simply create a new ArrowBuf associated + * with another combination of allocator buffer ledger for the same underlying memory + */ + + // the memory address stored inside ArrowBuf is its starting virtual + // address in the underlying memory chunk from the point it has + // access. so it is already accounting for the offset of the source buffer + // we can simply add the index to get the starting address of new buffer. + final long derivedBufferAddress = sourceBuffer.memoryAddress() + index; + + // create new ArrowBuf + final ArrowBuf derivedBuf = new ArrowBuf( + this, + null, + length, // length (in bytes) in the underlying memory chunk for this new ArrowBuf + derivedBufferAddress, // starting byte address in the underlying memory for this new ArrowBuf, + false); + + // logging + if (BaseAllocator.DEBUG) { + historicalLog.recordEvent( + "ArrowBuf(BufferLedger, BufferAllocator[%s], " + + "UnsafeDirectLittleEndian[identityHashCode == " + + "%d](%s)) => ledger hc == %d", + allocator.name, System.identityHashCode(derivedBuf), derivedBuf.toString(), + System.identityHashCode(this)); + + synchronized (buffers) { + buffers.put(derivedBuf, null); + } + } + + return derivedBuf; + } + + /** + * Used by an allocator to create a new ArrowBuf. This is provided + * as a helper method for the allocator when it allocates a new memory chunk + * using a new instance of allocation manager and creates a new reference manager + * too. + * + * @param length The length in bytes that this ArrowBuf will provide access to. + * @param manager An optional BufferManager argument that can be used to manage expansion of + * this ArrowBuf + * @return A new ArrowBuf that shares references with all ArrowBufs associated + * with this BufferLedger + */ + ArrowBuf newArrowBuf(final int length, final BufferManager manager) { + allocator.assertOpen(); + + // the start virtual address of the ArrowBuf will be same as address of memory chunk + final long startAddress = allocationManager.getMemoryChunk().memoryAddress(); + + // create ArrowBuf + final ArrowBuf buf = new ArrowBuf(this, manager, length, startAddress, false); + + // logging + if (BaseAllocator.DEBUG) { + historicalLog.recordEvent( + "ArrowBuf(BufferLedger, BufferAllocator[%s], " + + "UnsafeDirectLittleEndian[identityHashCode == " + "%d](%s)) => ledger hc == %d", + allocator.name, System.identityHashCode(buf), buf.toString(), + System.identityHashCode(this)); + + synchronized (buffers) { + buffers.put(buf, null); + } + } + + return buf; + } + + /** + * Create a new ArrowBuf that is associated with an alternative allocator for the purposes of + * memory ownership and accounting. This has no impact on the reference counting for the current + * ArrowBuf except in the situation where the passed in Allocator is the same as the current buffer. + *

+ * This operation has no impact on the reference count of this ArrowBuf. The newly created + * ArrowBuf with either have a reference count of 1 (in the case that this is the first time this + * memory is being associated with the target allocator or in other words allocation manager currently + * doesn't hold a mapping for the target allocator) or the current value of the reference count for + * the target allocator-reference manager combination + 1 in the case that the provided allocator + * already had an association to this underlying memory. + *

+ * + * @param srcBuffer source ArrowBuf + * @param target The target allocator to create an association with. + * @return A new ArrowBuf which shares the same underlying memory as the provided ArrowBuf. + */ + @Override + public ArrowBuf retain(final ArrowBuf srcBuffer, BufferAllocator target) { + if (srcBuffer.isEmpty()) { + return srcBuffer; + } + + if (BaseAllocator.DEBUG) { + historicalLog.recordEvent("retain(%s)", target.getName()); + } + + // the call to associate will return the corresponding reference manager (buffer ledger) for + // the target allocator. if the allocation manager didn't already have a mapping + // for the target allocator, it will create one and return the new reference manager with a + // reference count of 1. Thus the newly created buffer in this case will have a ref count of 1. + // alternatively, if there was already a mapping for in + // allocation manager, the ref count of the new buffer will be targetrefmanager.refcount() + 1 + // and this will be true for all the existing buffers currently managed by targetrefmanager + final BufferLedger targetRefManager = allocationManager.associate((BaseAllocator)target); + // create a new ArrowBuf to associate with new allocator and target ref manager + final int targetBufLength = srcBuffer.capacity(); + ArrowBuf targetArrowBuf = targetRefManager.deriveBuffer(srcBuffer, 0, targetBufLength); + targetArrowBuf.readerIndex(srcBuffer.readerIndex()); + targetArrowBuf.writerIndex(srcBuffer.writerIndex()); + return targetArrowBuf; + } + + /** + * Transfer any balance the current ledger has to the target ledger. In the case + * that the current ledger holds no memory, no transfer is made to the new ledger. + * + * @param targetReferenceManager The ledger to transfer ownership account to. + * @return Whether transfer fit within target ledgers limits. + */ + boolean transferBalance(final ReferenceManager targetReferenceManager) { + Preconditions.checkArgument(targetReferenceManager != null, + "Expecting valid target reference manager"); + final BaseAllocator targetAllocator = (BaseAllocator)targetReferenceManager.getAllocator(); + Preconditions.checkArgument(allocator.root == targetAllocator.root, + "You can only transfer between two allocators that share the same root."); + + allocator.assertOpen(); + targetReferenceManager.getAllocator().assertOpen(); + + // if we're transferring to ourself, just return. + if (targetReferenceManager == this) { + return true; + } + + // since two balance transfers out from the allocation manager could cause incorrect + // accounting, we need to ensure + // that this won't happen by synchronizing on the allocation manager instance. + synchronized (allocationManager) { + if (allocationManager.getOwningLedger() != this) { + // since the calling reference manager is not the owning + // reference manager for the underlying memory, transfer is + // a NO-OP + return true; + } + + if (BaseAllocator.DEBUG) { + this.historicalLog.recordEvent("transferBalance(%s)", + targetReferenceManager.getAllocator().getName()); + } + + boolean overlimit = targetAllocator.forceAllocate(allocationManager.getSize()); + allocator.releaseBytes(allocationManager.getSize()); + // since the transfer can only happen from the owning reference manager, + // we need to set the target ref manager as the new owning ref manager + // for the chunk of memory in allocation manager + allocationManager.setOwningLedger((BufferLedger)targetReferenceManager); + return overlimit; + } + } + + /** + * Transfer the memory accounting ownership of this ArrowBuf to another allocator. + * This will generate a new ArrowBuf that carries an association with the underlying memory + * of this ArrowBuf. If this ArrowBuf is connected to the owning BufferLedger of this memory, + * that memory ownership/accounting will be transferred to the target allocator. If this + * ArrowBuf does not currently own the memory underlying it (and is only associated with it), + * this does not transfer any ownership to the newly created ArrowBuf. + *

+ * This operation has no impact on the reference count of this ArrowBuf. The newly created + * ArrowBuf with either have a reference count of 1 (in the case that this is the first time + * this memory is being associated with the new allocator) or the current value of the reference + * count for the other AllocationManager/BufferLedger combination + 1 in the case that the provided + * allocator already had an association to this underlying memory. + *

+ *

+ * Transfers will always succeed, even if that puts the other allocator into an overlimit + * situation. This is possible due to the fact that the original owning allocator may have + * allocated this memory out of a local reservation whereas the target allocator may need to + * allocate new memory from a parent or RootAllocator. This operation is done n a mostly-lockless + * but consistent manner. As such, the overlimit==true situation could occur slightly prematurely + * to an actual overlimit==true condition. This is simply conservative behavior which means we may + * return overlimit slightly sooner than is necessary. + *

+ * + * @param target The allocator to transfer ownership to. + * @return A new transfer result with the impact of the transfer (whether it was overlimit) as + * well as the newly created ArrowBuf. + */ + @Override + public TransferResult transferOwnership(final ArrowBuf srcBuffer, final BufferAllocator target) { + if (srcBuffer.isEmpty()) { + return new TransferResult(true, srcBuffer); + } + // the call to associate will return the corresponding reference manager (buffer ledger) for + // the target allocator. if the allocation manager didn't already have a mapping + // for the target allocator, it will create one and return the new reference manager with a + // reference count of 1. Thus the newly created buffer in this case will have a ref count of 1. + // alternatively, if there was already a mapping for in + // allocation manager, the ref count of the new buffer will be targetrefmanager.refcount() + 1 + // and this will be true for all the existing buffers currently managed by targetrefmanager + final BufferLedger targetRefManager = allocationManager.associate((BaseAllocator)target); + // create a new ArrowBuf to associate with new allocator and target ref manager + final int targetBufLength = srcBuffer.capacity(); + final ArrowBuf targetArrowBuf = targetRefManager.deriveBuffer(srcBuffer, 0, targetBufLength); + targetArrowBuf.readerIndex(srcBuffer.readerIndex()); + targetArrowBuf.writerIndex(srcBuffer.writerIndex()); + final boolean allocationFit = transferBalance(targetRefManager); + return new TransferResult(allocationFit, targetArrowBuf); + } + + /** + * The outcome of a Transfer. + */ + public class TransferResult implements OwnershipTransferResult { + + // Whether this transfer fit within the target allocator's capacity. + final boolean allocationFit; + + // The newly created buffer associated with the target allocator + public final ArrowBuf buffer; + + private TransferResult(boolean allocationFit, ArrowBuf buffer) { + this.allocationFit = allocationFit; + this.buffer = buffer; + } + + @Override + public ArrowBuf getTransferredBuffer() { + return buffer; + } + + @Override + public boolean getAllocationFit() { + return allocationFit; + } + } + + /** + * Total size (in bytes) of memory underlying this reference manager. + * @return Size (in bytes) of the memory chunk + */ + @Override + public int getSize() { + return allocationManager.getSize(); + } + + /** + * How much memory is accounted for by this ledger. This is either getSize() + * if this is the owning ledger for the memory or zero in the case that this + * is not the owning ledger associated with this memory. + * @return Amount of accounted(owned) memory associated with this ledger. + */ + @Override + public int getAccountedSize() { + synchronized (allocationManager) { + if (allocationManager.getOwningLedger() == this) { + return allocationManager.getSize(); + } else { + return 0; + } + } + } + + /** + * Print the current ledger state to the provided StringBuilder. + * + * @param sb The StringBuilder to populate. + * @param indent The level of indentation to position the data. + * @param verbosity The level of verbosity to print. + */ + void print(StringBuilder sb, int indent, BaseAllocator.Verbosity verbosity) { + indent(sb, indent) + .append("ledger[") + .append(ledgerId) + .append("] allocator: ") + .append(allocator.name) + .append("), isOwning: ") + .append(", size: ") + .append(", references: ") + .append(bufRefCnt.get()) + .append(", life: ") + .append(lCreationTime) + .append("..") + .append(lDestructionTime) + .append(", allocatorManager: [") + .append(", life: "); + + if (!BaseAllocator.DEBUG) { + sb.append("]\n"); + } else { + synchronized (buffers) { + sb.append("] holds ") + .append(buffers.size()) + .append(" buffers. \n"); + for (ArrowBuf buf : buffers.keySet()) { + buf.print(sb, indent + 2, verbosity); + sb.append('\n'); + } + } + } + } + + public UnsafeDirectLittleEndian getUnderlying() { + return allocationManager.getMemoryChunk(); + } +} diff --git a/java/vector/src/main/java/org/apache/arrow/vector/VectorTrimmer.java b/java/memory/src/main/java/org/apache/arrow/memory/OwnershipTransferNOOP.java similarity index 69% rename from java/vector/src/main/java/org/apache/arrow/vector/VectorTrimmer.java rename to java/memory/src/main/java/org/apache/arrow/memory/OwnershipTransferNOOP.java index 9b3ccf170e5..7c0997d5396 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/VectorTrimmer.java +++ b/java/memory/src/main/java/org/apache/arrow/memory/OwnershipTransferNOOP.java @@ -15,19 +15,24 @@ * limitations under the License. */ -package org.apache.arrow.vector; +package org.apache.arrow.memory; import io.netty.buffer.ArrowBuf; -import io.netty.buffer.ByteBuf; -public class VectorTrimmer { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VectorTrimmer.class); +public class OwnershipTransferNOOP implements OwnershipTransferResult { + private final ArrowBuf buffer; - public static void trim(ByteBuf data, int idx) { - data.writerIndex(idx); - if (data instanceof ArrowBuf) { - // data.capacity(idx); - data.writerIndex(idx); - } + OwnershipTransferNOOP(final ArrowBuf buf) { + this.buffer = buf; + } + + @Override + public ArrowBuf getTransferredBuffer() { + return buffer; + } + + @Override + public boolean getAllocationFit() { + return true; } } diff --git a/java/memory/src/main/java/org/apache/arrow/memory/OwnershipTransferResult.java b/java/memory/src/main/java/org/apache/arrow/memory/OwnershipTransferResult.java new file mode 100644 index 00000000000..19255bdb504 --- /dev/null +++ b/java/memory/src/main/java/org/apache/arrow/memory/OwnershipTransferResult.java @@ -0,0 +1,27 @@ +/* + * 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.arrow.memory; + +import io.netty.buffer.ArrowBuf; + +public interface OwnershipTransferResult { + + boolean getAllocationFit(); + + ArrowBuf getTransferredBuffer(); +} diff --git a/java/memory/src/main/java/org/apache/arrow/memory/ReferenceManager.java b/java/memory/src/main/java/org/apache/arrow/memory/ReferenceManager.java new file mode 100644 index 00000000000..d131dfe0e67 --- /dev/null +++ b/java/memory/src/main/java/org/apache/arrow/memory/ReferenceManager.java @@ -0,0 +1,177 @@ +/* + * 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.arrow.memory; + + +import io.netty.buffer.ArrowBuf; + +/** + * Reference Manager manages one or more ArrowBufs that share the + * reference count for the underlying memory chunk. + */ +public interface ReferenceManager { + + /** + * Return the reference count. + * @return reference count + */ + int getRefCount(); + + /** + * Decrement this reference manager's reference count by 1 for the associated underlying + * memory. If the reference count drops to 0, it implies that ArrowBufs managed by this + * reference manager no longer need access to the underlying memory + * @return true if ref count has dropped to 0, false otherwise + */ + boolean release(); + + /** + * Decrement this reference manager's reference count for the associated underlying + * memory. If the reference count drops to 0, it implies that ArrowBufs managed by this + * reference manager no longer need access to the underlying memory + * @param decrement the count to decrease the reference count by + * @return the new reference count + */ + boolean release(int decrement); + + /** + * Increment this reference manager's reference count by 1 for the associated underlying + * memory. + */ + void retain(); + + /** + * Increment this reference manager's reference count by a given amount for the + * associated underlying memory. + * @param increment the count to increase the reference count by + */ + void retain(int increment); + + /** + * Create a new ArrowBuf that is associated with an alternative allocator for the purposes of + * memory ownership and accounting. This has no impact on the reference counting for the current + * ArrowBuf except in the situation where the passed in Allocator is the same as the current buffer. + * This operation has no impact on the reference count of this ArrowBuf. The newly created + * ArrowBuf with either have a reference count of 1 (in the case that this is the first time this + * memory is being associated with the target allocator or in other words allocation manager currently + * doesn't hold a mapping for the target allocator) or the current value of the reference count for + * the target allocator-reference manager combination + 1 in the case that the provided allocator + * already had an association to this underlying memory. + * + * @param srcBuffer source ArrowBuf + * @param targetAllocator The target allocator to create an association with. + * @return A new ArrowBuf which shares the same underlying memory as this ArrowBuf. + */ + ArrowBuf retain(ArrowBuf srcBuffer, BufferAllocator targetAllocator); + + /** + * Derive a new ArrowBuf from a given source ArrowBuf. The new derived + * ArrowBuf will share the same reference count as rest of the ArrowBufs + * associated with this reference manager. + * @param sourceBuffer source ArrowBuf + * @param index index (relative to source ArrowBuf) new ArrowBuf should be derived from + * @param length length (bytes) of data in underlying memory that derived buffer will + * have access to in underlying memory + * @return derived buffer + */ + ArrowBuf deriveBuffer(ArrowBuf sourceBuffer, int index, int length); + + /** + * Transfer the memory accounting ownership of this ArrowBuf to another allocator. + * This will generate a new ArrowBuf that carries an association with the underlying memory + * for the given ArrowBuf + * @param sourceBuffer source ArrowBuf + * @param targetAllocator The target allocator to create an association with + * @return {@link OwnershipTransferResult} with info on transfer result and new buffer + */ + OwnershipTransferResult transferOwnership(ArrowBuf sourceBuffer, BufferAllocator targetAllocator); + + /** + * Get the buffer allocator associated with this reference manager + * @return buffer allocator. + */ + BufferAllocator getAllocator(); + + /** + * Total size (in bytes) of memory underlying this reference manager. + * @return Size (in bytes) of the memory chunk. + */ + int getSize(); + + /** + * Get the total accounted size (in bytes). + * @return accounted size. + */ + int getAccountedSize(); + + public static String NO_OP_ERROR_MESSAGE = "Operation not supported on NO_OP Reference Manager"; + + // currently used for empty ArrowBufs + ReferenceManager NO_OP = new ReferenceManager() { + @Override + public int getRefCount() { + return 1; + } + + @Override + public boolean release() { + return false; + } + + @Override + public boolean release(int decrement) { + return false; + } + + @Override + public void retain() { } + + @Override + public void retain(int increment) { } + + @Override + public ArrowBuf retain(ArrowBuf srcBuffer, BufferAllocator targetAllocator) { + return srcBuffer; + } + + @Override + public ArrowBuf deriveBuffer(ArrowBuf sourceBuffer, int index, int length) { + return sourceBuffer; + } + + @Override + public OwnershipTransferResult transferOwnership(ArrowBuf sourceBuffer, BufferAllocator targetAllocator) { + return new OwnershipTransferNOOP(sourceBuffer); + } + + @Override + public BufferAllocator getAllocator() { + return new RootAllocator(0); + } + + @Override + public int getSize() { + return 0; + } + + @Override + public int getAccountedSize() { + return 0; + } + }; +} diff --git a/java/memory/src/test/java/org/apache/arrow/memory/TestBaseAllocator.java b/java/memory/src/test/java/org/apache/arrow/memory/TestBaseAllocator.java index 4da7434b52c..118fa556c85 100644 --- a/java/memory/src/test/java/org/apache/arrow/memory/TestBaseAllocator.java +++ b/java/memory/src/test/java/org/apache/arrow/memory/TestBaseAllocator.java @@ -27,7 +27,6 @@ import org.junit.Test; import io.netty.buffer.ArrowBuf; -import io.netty.buffer.ArrowBuf.TransferResult; public class TestBaseAllocator { @@ -67,10 +66,10 @@ public void test_privateMax() throws Exception { rootAllocator.newChildAllocator("noLimits", 0, MAX_ALLOCATION)) { final ArrowBuf arrowBuf2 = childAllocator.buffer(MAX_ALLOCATION / 2); assertNotNull("allocation failed", arrowBuf2); - arrowBuf2.release(); + arrowBuf2.getReferenceManager().release(); } - arrowBuf1.release(); + arrowBuf1.getReferenceManager().release(); } } @@ -104,7 +103,7 @@ public void testRootAllocator_getEmpty() throws Exception { final ArrowBuf arrowBuf = rootAllocator.buffer(0); assertNotNull("allocation failed", arrowBuf); assertEquals("capacity was non-zero", 0, arrowBuf.capacity()); - arrowBuf.release(); + arrowBuf.getReferenceManager().release(); } } @@ -129,17 +128,18 @@ public void testAllocator_transferOwnership() throws Exception { final ArrowBuf arrowBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 4); rootAllocator.verify(); - TransferResult transferOwnership = arrowBuf1.transferOwnership(childAllocator2); - assertEquiv(arrowBuf1, transferOwnership.buffer); - final boolean allocationFit = transferOwnership.allocationFit; + final ReferenceManager referenceManager = arrowBuf1.getReferenceManager(); + OwnershipTransferResult transferOwnership = referenceManager.transferOwnership(arrowBuf1, childAllocator2); + assertEquiv(arrowBuf1, transferOwnership.getTransferredBuffer()); + final boolean allocationFit = transferOwnership.getAllocationFit(); rootAllocator.verify(); assertTrue(allocationFit); - arrowBuf1.release(); + arrowBuf1.getReferenceManager().release(); childAllocator1.close(); rootAllocator.verify(); - transferOwnership.buffer.release(); + transferOwnership.getTransferredBuffer().getReferenceManager().release(); childAllocator2.close(); } } @@ -155,7 +155,7 @@ public void testAllocator_shareOwnership() throws Exception { rootAllocator.verify(); // share ownership of buffer. - final ArrowBuf arrowBuf2 = arrowBuf1.retain(childAllocator2); + final ArrowBuf arrowBuf2 = arrowBuf1.getReferenceManager().retain(arrowBuf1, childAllocator2); rootAllocator.verify(); assertNotNull(arrowBuf2); assertNotEquals(arrowBuf2, arrowBuf1); @@ -163,26 +163,26 @@ public void testAllocator_shareOwnership() throws Exception { // release original buffer (thus transferring ownership to allocator 2. (should leave // allocator 1 in empty state) - arrowBuf1.release(); + arrowBuf1.getReferenceManager().release(); rootAllocator.verify(); childAllocator1.close(); rootAllocator.verify(); final BufferAllocator childAllocator3 = rootAllocator.newChildAllocator("shareOwnership3", 0, MAX_ALLOCATION); - final ArrowBuf arrowBuf3 = arrowBuf1.retain(childAllocator3); + final ArrowBuf arrowBuf3 = arrowBuf1.getReferenceManager().retain(arrowBuf1, childAllocator3); assertNotNull(arrowBuf3); assertNotEquals(arrowBuf3, arrowBuf1); assertNotEquals(arrowBuf3, arrowBuf2); assertEquiv(arrowBuf1, arrowBuf3); rootAllocator.verify(); - arrowBuf2.release(); + arrowBuf2.getReferenceManager().release(); rootAllocator.verify(); childAllocator2.close(); rootAllocator.verify(); - arrowBuf3.release(); + arrowBuf3.getReferenceManager().release(); rootAllocator.verify(); childAllocator3.close(); } @@ -195,7 +195,7 @@ public void testRootAllocator_createChildAndUse() throws Exception { "createChildAndUse", 0, MAX_ALLOCATION)) { final ArrowBuf arrowBuf = childAllocator.buffer(512); assertNotNull("allocation failed", arrowBuf); - arrowBuf.release(); + arrowBuf.getReferenceManager().release(); } } } @@ -334,7 +334,7 @@ public void testRootAllocator_listeners() throws Exception { assertEquals(1, l1.getNumCalls()); assertEquals(0, l1.getNumReleaseCalls()); assertEquals(16, l1.getTotalMem()); - buf1.release(); + buf1.getReferenceManager().release(); try (final BufferAllocator c2 = c1.newChildAllocator("c2", l2, 0, MAX_ALLOCATION)) { assertEquals(2, l1.getNumChildren()); // c1 got a new child, so c1's listener (l1) is notified assertEquals(0, l2.getNumChildren()); @@ -346,7 +346,7 @@ public void testRootAllocator_listeners() throws Exception { assertEquals(1, l2.getNumCalls()); assertEquals(0, l2.getNumReleaseCalls()); assertEquals(32, l2.getTotalMem()); - buf2.release(); + buf2.getReferenceManager().release(); try (final BufferAllocator c3 = c2.newChildAllocator("c3", 0, MAX_ALLOCATION)) { assertEquals(2, l1.getNumChildren()); assertEquals(1, l2.getNumChildren()); @@ -360,7 +360,7 @@ public void testRootAllocator_listeners() throws Exception { assertEquals(2, l2.getNumCalls()); assertEquals(1, l2.getNumReleaseCalls()); assertEquals(32 + 64, l2.getTotalMem()); - buf3.release(); + buf3.getReferenceManager().release(); } assertEquals(2, l1.getNumChildren()); assertEquals(0, l2.getNumChildren()); // third-level child removed @@ -399,7 +399,7 @@ public void testRootAllocator_listenerAllocationFail() throws Exception { assertNotNull("allocation failed", arrowBuf); assertEquals(1, l1.getNumCalls()); assertEquals(MAX_ALLOCATION, l1.getTotalMem()); - arrowBuf.release(); + arrowBuf.getReferenceManager().release(); } } } @@ -407,11 +407,11 @@ public void testRootAllocator_listenerAllocationFail() throws Exception { private static void allocateAndFree(final BufferAllocator allocator) { final ArrowBuf arrowBuf = allocator.buffer(512); assertNotNull("allocation failed", arrowBuf); - arrowBuf.release(); + arrowBuf.getReferenceManager().release(); final ArrowBuf arrowBuf2 = allocator.buffer(MAX_ALLOCATION); assertNotNull("allocation failed", arrowBuf2); - arrowBuf2.release(); + arrowBuf2.getReferenceManager().release(); final int nBufs = 8; final ArrowBuf[] arrowBufs = new ArrowBuf[nBufs]; @@ -421,7 +421,7 @@ private static void allocateAndFree(final BufferAllocator allocator) { arrowBufs[i] = arrowBufi; } for (ArrowBuf arrowBufi : arrowBufs) { - arrowBufi.release(); + arrowBufi.getReferenceManager().release(); } } @@ -472,8 +472,8 @@ public void testAllocator_overAllocateParent() throws Exception { // expected } - arrowBuf1.release(); - arrowBuf2.release(); + arrowBuf1.getReferenceManager().release(); + arrowBuf2.getReferenceManager().release(); } } } @@ -491,7 +491,7 @@ private static void testAllocator_sliceUpBufferAndRelease( final ArrowBuf arrowBuf4 = arrowBuf3.slice(16, arrowBuf3.capacity() - 32); rootAllocator.verify(); - arrowBuf3.release(); // since they share refcounts, one is enough to release them all + arrowBuf3.getReferenceManager().release(); // since they share refcounts, one is enough to release them all rootAllocator.verify(); } @@ -516,7 +516,7 @@ public void testAllocator_createSlices() throws Exception { @SuppressWarnings("unused") final ArrowBuf arrowBuf2 = arrowBuf1.slice(MAX_ALLOCATION / 16, MAX_ALLOCATION / 16); testAllocator_sliceUpBufferAndRelease(rootAllocator, childAllocator); - arrowBuf1.release(); + arrowBuf1.getReferenceManager().release(); rootAllocator.verify(); } rootAllocator.verify(); @@ -582,7 +582,7 @@ public void testAllocator_sliceRanges() throws Exception { } */ - arrowBuf.release(); // all the derived buffers share this fate + arrowBuf.getReferenceManager().release(); // all the derived buffers share this fate } } @@ -618,7 +618,7 @@ public void testAllocator_slicesOfSlices() throws Exception { assertEquals((byte) i, slice30.getByte(i - 30)); } - arrowBuf.release(); + arrowBuf.getReferenceManager().release(); } } @@ -636,18 +636,18 @@ public void testAllocator_transferSliced() throws Exception { rootAllocator.verify(); - TransferResult result1 = arrowBuf2s.transferOwnership(childAllocator1); - assertEquiv(arrowBuf2s, result1.buffer); + OwnershipTransferResult result1 = arrowBuf2s.getReferenceManager().transferOwnership(arrowBuf2s, childAllocator1); + assertEquiv(arrowBuf2s, result1.getTransferredBuffer()); rootAllocator.verify(); - TransferResult result2 = arrowBuf1s.transferOwnership(childAllocator2); - assertEquiv(arrowBuf1s, result2.buffer); + OwnershipTransferResult result2 = arrowBuf1s.getReferenceManager().transferOwnership(arrowBuf1s, childAllocator2); + assertEquiv(arrowBuf1s, result2.getTransferredBuffer()); rootAllocator.verify(); - result1.buffer.release(); - result2.buffer.release(); + result1.getTransferredBuffer().getReferenceManager().release(); + result2.getTransferredBuffer().getReferenceManager().release(); - arrowBuf1s.release(); // releases arrowBuf1 - arrowBuf2s.release(); // releases arrowBuf2 + arrowBuf1s.getReferenceManager().release(); // releases arrowBuf1 + arrowBuf2s.getReferenceManager().release(); // releases arrowBuf2 childAllocator1.close(); childAllocator2.close(); @@ -668,18 +668,18 @@ public void testAllocator_shareSliced() throws Exception { rootAllocator.verify(); - final ArrowBuf arrowBuf2s1 = arrowBuf2s.retain(childAllocator1); + final ArrowBuf arrowBuf2s1 = arrowBuf2s.getReferenceManager().retain(arrowBuf2s, childAllocator1); assertEquiv(arrowBuf2s, arrowBuf2s1); - final ArrowBuf arrowBuf1s2 = arrowBuf1s.retain(childAllocator2); + final ArrowBuf arrowBuf1s2 = arrowBuf1s.getReferenceManager().retain(arrowBuf1s, childAllocator2); assertEquiv(arrowBuf1s, arrowBuf1s2); rootAllocator.verify(); - arrowBuf1s.release(); // releases arrowBuf1 - arrowBuf2s.release(); // releases arrowBuf2 + arrowBuf1s.getReferenceManager().release(); // releases arrowBuf1 + arrowBuf2s.getReferenceManager().release(); // releases arrowBuf2 rootAllocator.verify(); - arrowBuf2s1.release(); // releases the shared arrowBuf2 slice - arrowBuf1s2.release(); // releases the shared arrowBuf1 slice + arrowBuf2s1.getReferenceManager().release(); // releases the shared arrowBuf2 slice + arrowBuf1s2.getReferenceManager().release(); // releases the shared arrowBuf1 slice childAllocator1.close(); childAllocator2.close(); @@ -697,41 +697,43 @@ public void testAllocator_transferShared() throws Exception { boolean allocationFit; - ArrowBuf arrowBuf2 = arrowBuf1.retain(childAllocator2); + ArrowBuf arrowBuf2 = arrowBuf1.getReferenceManager().retain(arrowBuf1, childAllocator2); rootAllocator.verify(); assertNotNull(arrowBuf2); assertNotEquals(arrowBuf2, arrowBuf1); assertEquiv(arrowBuf1, arrowBuf2); - TransferResult result = arrowBuf1.transferOwnership(childAllocator3); - allocationFit = result.allocationFit; - final ArrowBuf arrowBuf3 = result.buffer; + final ReferenceManager refMananger1 = arrowBuf1.getReferenceManager(); + final OwnershipTransferResult result1 = refMananger1.transferOwnership(arrowBuf1, childAllocator3); + allocationFit = result1.getAllocationFit(); + final ArrowBuf arrowBuf3 = result1.getTransferredBuffer(); assertTrue(allocationFit); assertEquiv(arrowBuf1, arrowBuf3); rootAllocator.verify(); // Since childAllocator3 now has childAllocator1's buffer, 1, can close - arrowBuf1.release(); + arrowBuf1.getReferenceManager().release(); childAllocator1.close(); rootAllocator.verify(); - arrowBuf2.release(); + arrowBuf2.getReferenceManager().release(); childAllocator2.close(); rootAllocator.verify(); final BufferAllocator childAllocator4 = rootAllocator.newChildAllocator("transferShared4", 0, MAX_ALLOCATION); - TransferResult result2 = arrowBuf3.transferOwnership(childAllocator4); - allocationFit = result.allocationFit; - final ArrowBuf arrowBuf4 = result2.buffer; + final ReferenceManager refManager3 = arrowBuf3.getReferenceManager(); + final OwnershipTransferResult result3 = refManager3.transferOwnership(arrowBuf3, childAllocator4); + allocationFit = result3.getAllocationFit(); + final ArrowBuf arrowBuf4 = result3.getTransferredBuffer(); assertTrue(allocationFit); assertEquiv(arrowBuf3, arrowBuf4); rootAllocator.verify(); - arrowBuf3.release(); + arrowBuf3.getReferenceManager().release(); childAllocator3.close(); rootAllocator.verify(); - arrowBuf4.release(); + arrowBuf4.getReferenceManager().release(); childAllocator4.close(); rootAllocator.verify(); } @@ -765,7 +767,7 @@ public void testAllocator_claimedReservation() throws Exception { assertEquals(64, arrowBuf.capacity()); rootAllocator.verify(); - arrowBuf.release(); + arrowBuf.getReferenceManager().release(); rootAllocator.verify(); } rootAllocator.verify(); @@ -825,21 +827,21 @@ public void multiple() throws Exception { allocator.verify(); // Previously running operator completes - b22.release(); + b22.getReferenceManager().release(); allocator.verify(); allocator22.close(); - b31a.release(); + b31a.getReferenceManager().release(); allocator31.close(); - b12.release(); + b12.getReferenceManager().release(); allocator12.close(); allocator21.close(); - b11.release(); + b11.getReferenceManager().release(); allocator11.close(); frag1.close(); diff --git a/java/memory/src/test/java/org/apache/arrow/memory/TestEndianess.java b/java/memory/src/test/java/org/apache/arrow/memory/TestEndianess.java index 71568728cbe..2e32030c289 100644 --- a/java/memory/src/test/java/org/apache/arrow/memory/TestEndianess.java +++ b/java/memory/src/test/java/org/apache/arrow/memory/TestEndianess.java @@ -28,7 +28,7 @@ public class TestEndianess { @Test public void testLittleEndian() { final BufferAllocator a = new RootAllocator(10000); - final ByteBuf b = a.buffer(4); + final ByteBuf b = a.buffer(4).asNettyBuffer(); b.setInt(0, 35); assertEquals(b.getByte(0), 35); assertEquals(b.getByte(1), 0); diff --git a/java/pom.xml b/java/pom.xml index 9e4afcfa546..ffa1503a02d 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -381,6 +381,7 @@ true ${project.build.directory} + true diff --git a/java/vector/pom.xml b/java/vector/pom.xml index 66cd1cff441..b882e3efb2a 100644 --- a/java/vector/pom.xml +++ b/java/vector/pom.xml @@ -53,10 +53,6 @@ commons-codec 1.10 - - io.netty - netty-buffer - io.netty netty-common diff --git a/java/vector/src/main/codegen/templates/UnionVector.java b/java/vector/src/main/codegen/templates/UnionVector.java index 378ff60fa8e..1fabb6e5287 100644 --- a/java/vector/src/main/codegen/templates/UnionVector.java +++ b/java/vector/src/main/codegen/templates/UnionVector.java @@ -14,6 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +import io.netty.buffer.ArrowBuf; +import org.apache.arrow.memory.ReferenceManager; + <@pp.dropOutputFile /> <@pp.changeOutputFile name="/org/apache/arrow/vector/complex/UnionVector.java" /> @@ -109,8 +113,8 @@ public void loadFieldBuffers(ArrowFieldNode fieldNode, List ownBuffers } ArrowBuf buffer = ownBuffers.get(0); - typeBuffer.release(); - typeBuffer = buffer.retain(allocator); + typeBuffer.getReferenceManager().release(); + typeBuffer = buffer.getReferenceManager().retain(buffer, allocator); typeBufferAllocationSizeInBytes = typeBuffer.capacity(); this.valueCount = fieldNode.getLength(); } @@ -289,7 +293,7 @@ private void reallocTypeBuffer() { final ArrowBuf newBuf = allocator.buffer((int)newAllocationSize); newBuf.setBytes(0, typeBuffer, 0, currentBufferCapacity); newBuf.setZero(currentBufferCapacity, newBuf.capacity() - currentBufferCapacity); - typeBuffer.release(1); + typeBuffer.getReferenceManager().release(1); typeBuffer = newBuf; typeBufferAllocationSizeInBytes = (int)newAllocationSize; } @@ -310,7 +314,7 @@ public void close() { @Override public void clear() { valueCount = 0; - typeBuffer.release(); + typeBuffer.getReferenceManager().release(); typeBuffer = allocator.getEmpty(); internalStruct.clear(); } @@ -393,7 +397,8 @@ public TransferImpl(UnionVector to) { @Override public void transfer() { to.clear(); - to.typeBuffer = typeBuffer.transferOwnership(to.allocator).buffer; + final ReferenceManager refManager = typeBuffer.getReferenceManager(); + to.typeBuffer = refManager.transferOwnership(typeBuffer, to.allocator).getTransferredBuffer(); internalStructVectorTransferPair.transfer(); to.valueCount = valueCount; clear(); @@ -405,7 +410,9 @@ public void splitAndTransfer(int startIndex, int length) { internalStructVectorTransferPair.splitAndTransfer(startIndex, length); final int startPoint = startIndex * TYPE_WIDTH; final int sliceLength = length * TYPE_WIDTH; - to.typeBuffer = typeBuffer.slice(startPoint, sliceLength).transferOwnership(to.allocator).buffer; + final ArrowBuf slicedBuffer = typeBuffer.slice(startPoint, sliceLength); + final ReferenceManager refManager = slicedBuffer.getReferenceManager(); + to.typeBuffer = refManager.transferOwnership(slicedBuffer, to.allocator).getTransferredBuffer(); to.setValueCount(length); } @@ -466,8 +473,8 @@ public ArrowBuf[] getBuffers(boolean clear) { } if (clear) { valueCount = 0; - typeBuffer.retain(); - typeBuffer.release(); + typeBuffer.getReferenceManager().retain(); + typeBuffer.getReferenceManager().release(); typeBuffer = allocator.getEmpty(); } return list.toArray(new ArrowBuf[list.size()]); diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BaseFixedWidthVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BaseFixedWidthVector.java index fd1055dc687..7875c88304b 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/BaseFixedWidthVector.java +++ b/java/vector/src/main/java/org/apache/arrow/vector/BaseFixedWidthVector.java @@ -392,7 +392,7 @@ public ArrowBuf[] getBuffers(boolean clear) { } if (clear) { for (final ArrowBuf buffer : buffers) { - buffer.retain(1); + buffer.getReferenceManager().retain(1); } clear(); } @@ -419,13 +419,13 @@ public void reAlloc() { final ArrowBuf newValueBuffer = buffers.getDataBuf(); newValueBuffer.setBytes(0, valueBuffer, 0, valueBuffer.capacity()); newValueBuffer.setZero(valueBuffer.capacity(), newValueBuffer.capacity() - valueBuffer.capacity()); - valueBuffer.release(); + valueBuffer.getReferenceManager().release(); valueBuffer = newValueBuffer; final ArrowBuf newValidityBuffer = buffers.getValidityBuf(); newValidityBuffer.setBytes(0, validityBuffer, 0, validityBuffer.capacity()); newValidityBuffer.setZero(validityBuffer.capacity(), newValidityBuffer.capacity() - validityBuffer.capacity()); - validityBuffer.release(); + validityBuffer.getReferenceManager().release(); validityBuffer = newValidityBuffer; lastValueCapacity = getValueCapacity(); @@ -475,10 +475,10 @@ public void loadFieldBuffers(ArrowFieldNode fieldNode, List ownBuffers ArrowBuf bitBuffer = ownBuffers.get(0); ArrowBuf dataBuffer = ownBuffers.get(1); - validityBuffer.release(); + validityBuffer.getReferenceManager().release(); validityBuffer = BitVectorHelper.loadValidityBuffer(fieldNode, bitBuffer, allocator); - valueBuffer.release(); - valueBuffer = dataBuffer.retain(allocator); + valueBuffer.getReferenceManager().release(); + valueBuffer = dataBuffer.getReferenceManager().retain(dataBuffer, allocator); valueCount = fieldNode.getLength(); } @@ -555,8 +555,8 @@ public TransferPair getTransferPair(BufferAllocator allocator) { public void transferTo(BaseFixedWidthVector target) { compareTypes(target, "transferTo"); target.clear(); - target.validityBuffer = validityBuffer.transferOwnership(target.allocator).buffer; - target.valueBuffer = valueBuffer.transferOwnership(target.allocator).buffer; + target.validityBuffer = transferBuffer(validityBuffer, target.allocator); + target.valueBuffer = transferBuffer(valueBuffer, target.allocator); target.valueCount = valueCount; clear(); } @@ -584,7 +584,8 @@ private void splitAndTransferValueBuffer(int startIndex, int length, BaseFixedWidthVector target) { final int startPoint = startIndex * typeWidth; final int sliceLength = length * typeWidth; - target.valueBuffer = valueBuffer.slice(startPoint, sliceLength).transferOwnership(target.allocator).buffer; + final ArrowBuf slicedBuffer = valueBuffer.slice(startPoint, sliceLength); + target.valueBuffer = transferBuffer(slicedBuffer, target.allocator); } /** @@ -603,10 +604,10 @@ private void splitAndTransferValidityBuffer(int startIndex, int length, if (offset == 0) { /* slice */ if (target.validityBuffer != null) { - target.validityBuffer.release(); + target.validityBuffer.getReferenceManager().release(); } target.validityBuffer = validityBuffer.slice(firstByteSource, byteSizeTarget); - target.validityBuffer.retain(1); + target.validityBuffer.getReferenceManager().retain(1); } else { /* Copy data * When the first bit starts from the middle of a byte (offset != 0), diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BaseValueVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BaseValueVector.java index 4e014bbd2ae..ade31b75ff8 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/BaseValueVector.java +++ b/java/vector/src/main/java/org/apache/arrow/vector/BaseValueVector.java @@ -22,6 +22,7 @@ import org.apache.arrow.memory.BaseAllocator; import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.ReferenceManager; import org.apache.arrow.util.Preconditions; import org.apache.arrow.vector.util.TransferPair; import org.slf4j.Logger; @@ -90,14 +91,14 @@ public BufferAllocator getAllocator() { return allocator; } - protected void compareTypes(BaseValueVector target, String caller) { + void compareTypes(BaseValueVector target, String caller) { if (this.getMinorType() != target.getMinorType()) { throw new UnsupportedOperationException(caller + " should have vectors of exact same type"); } } protected ArrowBuf releaseBuffer(ArrowBuf buffer) { - buffer.release(); + buffer.getReferenceManager().release(); buffer = allocator.getEmpty(); return buffer; } @@ -112,7 +113,7 @@ private static long roundUp8(long size) { return ((size + 7) / 8) * 8; } - protected long computeCombinedBufferSize(int valueCount, int typeWidth) { + long computeCombinedBufferSize(int valueCount, int typeWidth) { Preconditions.checkArgument(valueCount >= 0, "valueCount must be >= 0"); Preconditions.checkArgument(typeWidth >= 0, "typeWidth must be >= 0"); @@ -138,17 +139,16 @@ class DataAndValidityBuffers { this.validityBuf = validityBuf; } - public ArrowBuf getDataBuf() { + ArrowBuf getDataBuf() { return dataBuf; } - public ArrowBuf getValidityBuf() { + ArrowBuf getValidityBuf() { return validityBuf; } - } - protected DataAndValidityBuffers allocFixedDataAndValidityBufs(int valueCount, int typeWidth) { + DataAndValidityBuffers allocFixedDataAndValidityBufs(int valueCount, int typeWidth) { long bufferSize = computeCombinedBufferSize(valueCount, typeWidth); assert bufferSize < MAX_ALLOCATION_SIZE; @@ -181,7 +181,7 @@ protected DataAndValidityBuffers allocFixedDataAndValidityBufs(int valueCount, i for (int numBuffers = 0; numBuffers < 2; ++numBuffers) { int len = (numBuffers == 0 ? dataBufferSize : validityBufferSize); ArrowBuf buf = combinedBuffer.slice(bufferOffset, len); - buf.retain(); + buf.getReferenceManager().retain(); buf.readerIndex(0); buf.writerIndex(0); @@ -192,8 +192,13 @@ protected DataAndValidityBuffers allocFixedDataAndValidityBufs(int valueCount, i validityBuf = buf; } } - combinedBuffer.release(); + combinedBuffer.getReferenceManager().release(); return new DataAndValidityBuffers(dataBuf, validityBuf); } + + public static ArrowBuf transferBuffer(final ArrowBuf srcBuffer, final BufferAllocator targetAllocator) { + final ReferenceManager referenceManager = srcBuffer.getReferenceManager(); + return referenceManager.transferOwnership(srcBuffer, targetAllocator).getTransferredBuffer(); + } } diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BaseVariableWidthVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BaseVariableWidthVector.java index 9d765a468b4..99dcc39add4 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/BaseVariableWidthVector.java +++ b/java/vector/src/main/java/org/apache/arrow/vector/BaseVariableWidthVector.java @@ -295,12 +295,12 @@ public void loadFieldBuffers(ArrowFieldNode fieldNode, List ownBuffers ArrowBuf offBuffer = ownBuffers.get(1); ArrowBuf dataBuffer = ownBuffers.get(2); - validityBuffer.release(); + validityBuffer.getReferenceManager().release(); validityBuffer = BitVectorHelper.loadValidityBuffer(fieldNode, bitBuffer, allocator); - offsetBuffer.release(); - offsetBuffer = offBuffer.retain(allocator); - valueBuffer.release(); - valueBuffer = dataBuffer.retain(allocator); + offsetBuffer.getReferenceManager().release(); + offsetBuffer = offBuffer.getReferenceManager().retain(offBuffer, allocator); + valueBuffer.getReferenceManager().release(); + valueBuffer = dataBuffer.getReferenceManager().retain(dataBuffer, allocator); lastSet = fieldNode.getLength() - 1; valueCount = fieldNode.getLength(); @@ -493,7 +493,7 @@ public void reallocDataBuffer() { final ArrowBuf newBuf = allocator.buffer((int) newAllocationSize); newBuf.setBytes(0, valueBuffer, 0, currentBufferCapacity); - valueBuffer.release(); + valueBuffer.getReferenceManager().release(); valueBuffer = newBuf; lastValueAllocationSizeInBytes = valueBuffer.capacity(); } @@ -536,13 +536,13 @@ public void reallocValidityAndOffsetBuffers() { final ArrowBuf newOffsetBuffer = buffers.getDataBuf(); newOffsetBuffer.setBytes(0, offsetBuffer, 0, offsetBuffer.capacity()); newOffsetBuffer.setZero(offsetBuffer.capacity(), newOffsetBuffer.capacity() - offsetBuffer.capacity()); - offsetBuffer.release(); + offsetBuffer.getReferenceManager().release(); offsetBuffer = newOffsetBuffer; final ArrowBuf newValidityBuffer = buffers.getValidityBuf(); newValidityBuffer.setBytes(0, validityBuffer, 0, validityBuffer.capacity()); newValidityBuffer.setZero(validityBuffer.capacity(), newValidityBuffer.capacity() - validityBuffer.capacity()); - validityBuffer.release(); + validityBuffer.getReferenceManager().release(); validityBuffer = newValidityBuffer; lastValueCapacity = getValueCapacity(); @@ -626,7 +626,7 @@ public ArrowBuf[] getBuffers(boolean clear) { } if (clear) { for (final ArrowBuf buffer : buffers) { - buffer.retain(1); + buffer.getReferenceManager().retain(); } clear(); } @@ -672,9 +672,9 @@ public TransferPair getTransferPair(BufferAllocator allocator) { public void transferTo(BaseVariableWidthVector target) { compareTypes(target, "transferTo"); target.clear(); - target.validityBuffer = validityBuffer.transferOwnership(target.allocator).buffer; - target.valueBuffer = valueBuffer.transferOwnership(target.allocator).buffer; - target.offsetBuffer = offsetBuffer.transferOwnership(target.allocator).buffer; + target.validityBuffer = transferBuffer(validityBuffer, target.allocator); + target.valueBuffer = transferBuffer(valueBuffer, target.allocator); + target.offsetBuffer = transferBuffer(offsetBuffer, target.allocator); target.setLastSet(this.lastSet); if (this.valueCount > 0) { target.setValueCount(this.valueCount); @@ -718,7 +718,8 @@ private void splitAndTransferOffsetBuffer(int startIndex, int length, BaseVariab final int relativeSourceOffset = offsetBuffer.getInt((startIndex + i) * OFFSET_WIDTH) - start; target.offsetBuffer.setInt(i * OFFSET_WIDTH, relativeSourceOffset); } - target.valueBuffer = valueBuffer.slice(start, dataLength).transferOwnership(target.allocator).buffer; + final ArrowBuf slicedBuffer = valueBuffer.slice(start, dataLength); + target.valueBuffer = transferBuffer(slicedBuffer, target.allocator); } /* @@ -736,10 +737,10 @@ private void splitAndTransferValidityBuffer(int startIndex, int length, if (offset == 0) { // slice if (target.validityBuffer != null) { - target.validityBuffer.release(); + target.validityBuffer.getReferenceManager().release(); } target.validityBuffer = validityBuffer.slice(firstByteSource, byteSizeTarget); - target.validityBuffer.retain(1); + target.validityBuffer.getReferenceManager().retain(); } else { /* Copy data * When the first bit starts from the middle of a byte (offset != 0), diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java index b806ec7205b..c5b5be47c81 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java +++ b/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java @@ -168,10 +168,10 @@ private ArrowBuf splitAndTransferBuffer( if (offset == 0) { /* slice */ if (destBuffer != null) { - destBuffer.release(); + destBuffer.getReferenceManager().release(); } destBuffer = sourceBuffer.slice(firstByteSource, byteSizeTarget); - destBuffer.retain(1); + destBuffer.getReferenceManager().retain(1); } else { /* Copy data * When the first bit starts from the middle of a byte (offset != 0), diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BitVectorHelper.java b/java/vector/src/main/java/org/apache/arrow/vector/BitVectorHelper.java index 2fb3f6b94c9..ec15cc4432b 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/BitVectorHelper.java +++ b/java/vector/src/main/java/org/apache/arrow/vector/BitVectorHelper.java @@ -196,7 +196,7 @@ public static ArrowBuf loadValidityBuffer(final ArrowFieldNode fieldNode, /* mixed byte pattern -- create another ArrowBuf associated with the * target allocator */ - newBuffer = sourceValidityBuffer.retain(allocator); + newBuffer = sourceValidityBuffer.getReferenceManager().retain(sourceValidityBuffer, allocator); } return newBuffer; diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/AbstractStructVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/AbstractStructVector.java index 321ad8a9171..bbd3ff28ebd 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/complex/AbstractStructVector.java +++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/AbstractStructVector.java @@ -290,7 +290,7 @@ public ArrowBuf[] getBuffers(boolean clear) { for (final ArrowBuf buf : vector.getBuffers(false)) { buffers.add(buf); if (clear) { - buf.retain(1); + buf.getReferenceManager().retain(1); } } if (clear) { diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/BaseRepeatedValueVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/BaseRepeatedValueVector.java index 4c61dfb493c..34ba61d2a64 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/complex/BaseRepeatedValueVector.java +++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/BaseRepeatedValueVector.java @@ -118,7 +118,7 @@ protected void reallocOffsetBuffer() { final ArrowBuf newBuf = allocator.buffer((int) newAllocationSize); newBuf.setBytes(0, offsetBuffer, 0, currentBufferCapacity); newBuf.setZero(currentBufferCapacity, newBuf.capacity() - currentBufferCapacity); - offsetBuffer.release(1); + offsetBuffer.getReferenceManager().release(1); offsetBuffer = newBuf; offsetAllocationSizeInBytes = (int) newAllocationSize; } @@ -247,7 +247,7 @@ public ArrowBuf[] getBuffers(boolean clear) { } if (clear) { for (ArrowBuf buffer : buffers) { - buffer.retain(); + buffer.getReferenceManager().retain(); } clear(); } diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/FixedSizeListVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/FixedSizeListVector.java index 31bdf14f4ba..16feef9bac7 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/complex/FixedSizeListVector.java +++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/FixedSizeListVector.java @@ -135,7 +135,7 @@ public void loadFieldBuffers(ArrowFieldNode fieldNode, List ownBuffers ArrowBuf bitBuffer = ownBuffers.get(0); - validityBuffer.release(); + validityBuffer.getReferenceManager().release(); validityBuffer = BitVectorHelper.loadValidityBuffer(fieldNode, bitBuffer, allocator); valueCount = fieldNode.getLength(); @@ -231,7 +231,7 @@ private void reallocValidityBuffer() { final ArrowBuf newBuf = allocator.buffer((int) newAllocationSize); newBuf.setBytes(0, validityBuffer, 0, currentBufferCapacity); newBuf.setZero(currentBufferCapacity, newBuf.capacity() - currentBufferCapacity); - validityBuffer.release(1); + validityBuffer.getReferenceManager().release(1); validityBuffer = newBuf; validityAllocationSizeInBytes = (int) newAllocationSize; } @@ -305,7 +305,7 @@ public ArrowBuf[] getBuffers(boolean clear) { } if (clear) { for (ArrowBuf buffer : buffers) { - buffer.retain(); + buffer.getReferenceManager().retain(); } clear(); } @@ -481,7 +481,7 @@ public TransferImpl(FixedSizeListVector to) { public void transfer() { to.clear(); dataPair.transfer(); - to.validityBuffer = validityBuffer.transferOwnership(to.allocator).buffer; + to.validityBuffer = BaseValueVector.transferBuffer(validityBuffer, to.allocator); to.setValueCount(valueCount); clear(); } diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java index 7d98f71477d..932e7d386e7 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java +++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java @@ -171,10 +171,10 @@ public void loadFieldBuffers(ArrowFieldNode fieldNode, List ownBuffers ArrowBuf bitBuffer = ownBuffers.get(0); ArrowBuf offBuffer = ownBuffers.get(1); - validityBuffer.release(); + validityBuffer.getReferenceManager().release(); validityBuffer = BitVectorHelper.loadValidityBuffer(fieldNode, bitBuffer, allocator); - offsetBuffer.release(); - offsetBuffer = offBuffer.retain(allocator); + offsetBuffer.getReferenceManager().release(); + offsetBuffer = offBuffer.getReferenceManager().retain(offBuffer, allocator); validityAllocationSizeInBytes = validityBuffer.capacity(); offsetAllocationSizeInBytes = offsetBuffer.capacity(); @@ -301,7 +301,7 @@ private void reallocValidityBuffer() { final ArrowBuf newBuf = allocator.buffer((int) newAllocationSize); newBuf.setBytes(0, validityBuffer, 0, currentBufferCapacity); newBuf.setZero(currentBufferCapacity, newBuf.capacity() - currentBufferCapacity); - validityBuffer.release(1); + validityBuffer.getReferenceManager().release(1); validityBuffer = newBuf; validityAllocationSizeInBytes = (int) newAllocationSize; } @@ -414,8 +414,8 @@ public TransferImpl(ListVector to) { public void transfer() { to.clear(); dataTransferPair.transfer(); - to.validityBuffer = validityBuffer.transferOwnership(to.allocator).buffer; - to.offsetBuffer = offsetBuffer.transferOwnership(to.allocator).buffer; + to.validityBuffer = transferBuffer(validityBuffer, to.allocator); + to.offsetBuffer = transferBuffer(offsetBuffer, to.allocator); to.lastSet = lastSet; if (valueCount > 0) { to.setValueCount(valueCount); @@ -462,10 +462,10 @@ private void splitAndTransferValidityBuffer(int startIndex, int length, ListVect if (offset == 0) { // slice if (target.validityBuffer != null) { - target.validityBuffer.release(); + target.validityBuffer.getReferenceManager().release(); } target.validityBuffer = validityBuffer.slice(firstByteSource, byteSizeTarget); - target.validityBuffer.retain(1); + target.validityBuffer.getReferenceManager().retain(1); } else { /* Copy data * When the first bit starts from the middle of a byte (offset != 0), @@ -594,7 +594,7 @@ public ArrowBuf[] getBuffers(boolean clear) { } if (clear) { for (ArrowBuf buffer : buffers) { - buffer.retain(); + buffer.getReferenceManager().retain(); } clear(); } diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/StructVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/StructVector.java index bc281df5973..b630a90132d 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/complex/StructVector.java +++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/StructVector.java @@ -92,7 +92,7 @@ public void loadFieldBuffers(ArrowFieldNode fieldNode, List ownBuffers ArrowBuf bitBuffer = ownBuffers.get(0); - validityBuffer.release(); + validityBuffer.getReferenceManager().release(); validityBuffer = BitVectorHelper.loadValidityBuffer(fieldNode, bitBuffer, allocator); valueCount = fieldNode.getLength(); validityAllocationSizeInBytes = validityBuffer.capacity(); @@ -159,7 +159,7 @@ protected NullableStructTransferPair(StructVector from, StructVector to, boolean @Override public void transfer() { target.clear(); - target.validityBuffer = validityBuffer.transferOwnership(target.allocator).buffer; + target.validityBuffer = BaseValueVector.transferBuffer(validityBuffer, target.allocator); super.transfer(); clear(); } @@ -195,10 +195,10 @@ private void splitAndTransferValidityBuffer(int startIndex, int length, StructVe if (offset == 0) { // slice if (target.validityBuffer != null) { - target.validityBuffer.release(); + target.validityBuffer.getReferenceManager().release(); } target.validityBuffer = validityBuffer.slice(firstByteSource, byteSizeTarget); - target.validityBuffer.retain(1); + target.validityBuffer.getReferenceManager().retain(1); } else { /* Copy data * When the first bit starts from the middle of a byte (offset != 0), @@ -283,7 +283,7 @@ public ArrowBuf[] getBuffers(boolean clear) { } if (clear) { for (ArrowBuf buffer : buffers) { - buffer.retain(); + buffer.getReferenceManager().retain(); } clear(); } @@ -322,7 +322,7 @@ public void reset() { * Release the validity buffer. */ private void clearValidityBuffer() { - validityBuffer.release(); + validityBuffer.getReferenceManager().release(); validityBuffer = allocator.getEmpty(); } @@ -423,7 +423,7 @@ private void reallocValidityBuffer() { final ArrowBuf newBuf = allocator.buffer((int) newAllocationSize); newBuf.setBytes(0, validityBuffer, 0, currentBufferCapacity); newBuf.setZero(currentBufferCapacity, newBuf.capacity() - currentBufferCapacity); - validityBuffer.release(1); + validityBuffer.getReferenceManager().release(1); validityBuffer = newBuf; validityAllocationSizeInBytes = (int) newAllocationSize; } diff --git a/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileReader.java b/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileReader.java index f5948777cd0..e5fca8cc442 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileReader.java +++ b/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileReader.java @@ -576,7 +576,7 @@ private void readFromJsonIntoVector(Field field, FieldVector vector) throws Json readToken(END_OBJECT); for (ArrowBuf buffer: vectorBuffers) { - buffer.release(); + buffer.getReferenceManager().release(); } } diff --git a/java/vector/src/main/java/org/apache/arrow/vector/ipc/message/ArrowRecordBatch.java b/java/vector/src/main/java/org/apache/arrow/vector/ipc/message/ArrowRecordBatch.java index 26d39165f80..ce7aefc3c2e 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/ipc/message/ArrowRecordBatch.java +++ b/java/vector/src/main/java/org/apache/arrow/vector/ipc/message/ArrowRecordBatch.java @@ -71,7 +71,7 @@ public ArrowRecordBatch(int length, List nodes, List b List arrowBuffers = new ArrayList<>(); long offset = 0; for (ArrowBuf arrowBuf : buffers) { - arrowBuf.retain(); + arrowBuf.getReferenceManager().retain(); long size = arrowBuf.readableBytes(); arrowBuffers.add(new ArrowBuffer(offset, size)); LOGGER.debug("Buffer in RecordBatch at {}, length: {}", offset, size); @@ -134,7 +134,10 @@ public List getBuffers() { */ public ArrowRecordBatch cloneWithTransfer(final BufferAllocator allocator) { final List newBufs = buffers.stream() - .map(t -> (t.transferOwnership(allocator).buffer).writerIndex(t.writerIndex())) + .map(buf -> + (buf.getReferenceManager().transferOwnership(buf, allocator) + .getTransferredBuffer()) + .writerIndex(buf.writerIndex())) .collect(Collectors.toList()); close(); return new ArrowRecordBatch(false, length, nodes, newBufs); @@ -175,7 +178,7 @@ public void close() { if (!closed) { closed = true; for (ArrowBuf arrowBuf : buffers) { - arrowBuf.release(); + arrowBuf.getReferenceManager().release(); } } } diff --git a/java/vector/src/main/java/org/apache/arrow/vector/ipc/message/MessageSerializer.java b/java/vector/src/main/java/org/apache/arrow/vector/ipc/message/MessageSerializer.java index 1449846eb65..034a34c8f1d 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/ipc/message/MessageSerializer.java +++ b/java/vector/src/main/java/org/apache/arrow/vector/ipc/message/MessageSerializer.java @@ -336,7 +336,7 @@ public static ArrowRecordBatch deserializeRecordBatch(RecordBatch recordBatchFB, } ArrowRecordBatch arrowRecordBatch = new ArrowRecordBatch((int) recordBatchFB.length(), nodes, buffers); - body.release(); + body.getReferenceManager().release(); return arrowRecordBatch; } diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestBitVectorHelper.java b/java/vector/src/test/java/org/apache/arrow/vector/TestBitVectorHelper.java index c66f512c606..f62371d7525 100644 --- a/java/vector/src/test/java/org/apache/arrow/vector/TestBitVectorHelper.java +++ b/java/vector/src/test/java/org/apache/arrow/vector/TestBitVectorHelper.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; +import org.apache.arrow.memory.ReferenceManager; import org.junit.Test; import io.netty.buffer.ArrowBuf; @@ -29,8 +30,7 @@ public class TestBitVectorHelper { public void testGetNullCount() throws Exception { // test case 1, 1 null value for 0b110 ArrowBuf validityBuffer = new ArrowBuf( - null, null, new PooledByteBufAllocatorL().empty, - null, null, 0, 3, true); + ReferenceManager.NO_OP, null,3, new PooledByteBufAllocatorL().empty.memoryAddress(), true); // we set validity buffer to be 0b10110, but only have 3 items with 1st item is null validityBuffer.setByte(0, 0b10110); @@ -40,8 +40,7 @@ null, null, new PooledByteBufAllocatorL().empty, // test case 2, no null value for 0xFF validityBuffer = new ArrowBuf( - null, null, new PooledByteBufAllocatorL().empty, - null, null, 0, 8, true); + ReferenceManager.NO_OP, null,8, new PooledByteBufAllocatorL().empty.memoryAddress(), true); validityBuffer.setByte(0, 0xFF); count = BitVectorHelper.getNullCount(validityBuffer, 8); @@ -49,8 +48,7 @@ null, null, new PooledByteBufAllocatorL().empty, // test case 3, 1 null value for 0x7F validityBuffer = new ArrowBuf( - null, null, new PooledByteBufAllocatorL().empty, - null, null, 0, 8, true); + ReferenceManager.NO_OP, null, 8, new PooledByteBufAllocatorL().empty.memoryAddress(), true); validityBuffer.setByte(0, 0x7F); count = BitVectorHelper.getNullCount(validityBuffer, 8); @@ -58,8 +56,7 @@ null, null, new PooledByteBufAllocatorL().empty, // test case 4, validity buffer has multiple bytes, 11 items validityBuffer = new ArrowBuf( - null, null, new PooledByteBufAllocatorL().empty, - null, null, 0, 11, true); + ReferenceManager.NO_OP, null,11, new PooledByteBufAllocatorL().empty.memoryAddress(), true); validityBuffer.setByte(0, 0b10101010); validityBuffer.setByte(1, 0b01010101); diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestBufferOwnershipTransfer.java b/java/vector/src/test/java/org/apache/arrow/vector/TestBufferOwnershipTransfer.java index a407166c4f6..10dd11a4813 100644 --- a/java/vector/src/test/java/org/apache/arrow/vector/TestBufferOwnershipTransfer.java +++ b/java/vector/src/test/java/org/apache/arrow/vector/TestBufferOwnershipTransfer.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue; import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.ReferenceManager; import org.apache.arrow.memory.RootAllocator; import org.apache.arrow.vector.complex.ListVector; import org.apache.arrow.vector.types.pojo.ArrowType; @@ -97,7 +98,16 @@ public void emptyListTransferShouldNotTriggerSchemaChange() { FieldType.nullable(ArrowType.Null.INSTANCE), newTriggerCallback(trigger2)); - v1.makeTransferPair(v2).transfer(); + try { + // since we are working with empty vectors, their internal + // buffers will be allocator.EMPTY which use + // ReferenceManager.NO_OP instance and transfer() is not + // supported + v1.makeTransferPair(v2).transfer(); + } catch (Exception e) { + assertTrue(e instanceof UnsupportedOperationException); + assertTrue(e.getMessage().contains(ReferenceManager.NO_OP_ERROR_MESSAGE)); + } assertFalse(trigger1.value); assertFalse(trigger2.value); diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestVectorUnloadLoad.java b/java/vector/src/test/java/org/apache/arrow/vector/TestVectorUnloadLoad.java index cbba73441cb..dce96ce41c2 100644 --- a/java/vector/src/test/java/org/apache/arrow/vector/TestVectorUnloadLoad.java +++ b/java/vector/src/test/java/org/apache/arrow/vector/TestVectorUnloadLoad.java @@ -183,7 +183,7 @@ public void testUnloadLoadAddPadding() throws IOException { } for (ArrowBuf newBuf : newBuffers) { - newBuf.release(); + newBuf.getReferenceManager().release(); } } } @@ -269,7 +269,7 @@ public void testLoadValidityBuffer() throws IOException { assertEquals(1234, intDefinedVector.get(count + 10)); } finally { for (ArrowBuf arrowBuf : values) { - arrowBuf.release(); + arrowBuf.getReferenceManager().release(); } } } diff --git a/java/vector/src/test/java/org/apache/arrow/vector/ipc/BaseFileTest.java b/java/vector/src/test/java/org/apache/arrow/vector/ipc/BaseFileTest.java index e72429376e3..447ed703458 100644 --- a/java/vector/src/test/java/org/apache/arrow/vector/ipc/BaseFileTest.java +++ b/java/vector/src/test/java/org/apache/arrow/vector/ipc/BaseFileTest.java @@ -151,7 +151,7 @@ protected void writeComplexData(int count, StructVector parent) { structWriter.end(); } writer.setValueCount(count); - varchar.release(); + varchar.getReferenceManager().release(); } public void printVectors(List vectors) { @@ -554,7 +554,7 @@ public void writeUnionData(int count, StructVector parent) { } } writer.setValueCount(count); - varchar.release(); + varchar.getReferenceManager().release(); } protected void writeVarBinaryData(int count, StructVector parent) { @@ -573,7 +573,7 @@ protected void writeVarBinaryData(int count, StructVector parent) { listWriter.endList(); } writer.setValueCount(count); - varbin.release(); + varbin.getReferenceManager().release(); } protected void validateVarBinary(int count, VectorSchemaRoot root) { diff --git a/python/pyarrow/jvm.py b/python/pyarrow/jvm.py index 28b72711764..2341f405403 100644 --- a/python/pyarrow/jvm.py +++ b/python/pyarrow/jvm.py @@ -46,7 +46,7 @@ def jvm_buffer(arrowbuf): """ address = arrowbuf.memoryAddress() size = arrowbuf.capacity() - return pa.foreign_buffer(address, size, arrowbuf.unwrap()) + return pa.foreign_buffer(address, size, arrowbuf.asNettyBuffer()) def _from_jvm_int_type(jvm_type):