Skip to content

Commit fac0840

Browse files
authored
ARROW-16674: [Java] C data interface: Reading as nioBuffer from imported buffer causes error (apache#13249)
Lead-authored-by: Hongze Zhang <hongze.zhang@intel.com> Co-authored-by: Hongze Zhang <mailtozhz@126.com> Signed-off-by: Antoine Pitrou <antoine@python.org>
1 parent 8ec29ce commit fac0840

3 files changed

Lines changed: 32 additions & 14 deletions

File tree

java/c/src/main/java/org/apache/arrow/c/ArrayImporter.java

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -141,9 +141,10 @@ private List<ArrowBuf> importBuffers(ArrowArray.Snapshot snapshot) {
141141
for (long bufferPtr : buffers) {
142142
ArrowBuf buffer = null;
143143
if (bufferPtr != NULL) {
144-
// TODO(roee88): an API for getting the size for each buffer is not yet
145-
// available
146-
buffer = new ArrowBuf(referenceManager, null, Integer.MAX_VALUE, bufferPtr);
144+
// See ARROW-17720: [Java] C data interface: Add API to compute imported buffer size
145+
int capacity = Integer.MAX_VALUE;
146+
buffer = new ArrowBuf(referenceManager, null, capacity, bufferPtr);
147+
buffer.writerIndex(capacity);
147148
}
148149
result.add(buffer);
149150
}

java/c/src/test/java/org/apache/arrow/c/RoundtripTest.java

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import static org.junit.jupiter.api.Assertions.assertTrue;
2424

2525
import java.nio.ByteBuffer;
26+
import java.nio.ByteOrder;
2627
import java.nio.charset.StandardCharsets;
2728
import java.util.ArrayList;
2829
import java.util.Arrays;
@@ -715,6 +716,22 @@ public void testSchema() {
715716
}
716717
}
717718

719+
@Test
720+
public void testImportedBufferAsNioBuffer() {
721+
IntVector imported;
722+
try (final IntVector vector = new IntVector("v", allocator)) {
723+
setVector(vector, 1, 2, 3, null);
724+
imported = (IntVector) vectorRoundtrip(vector);
725+
}
726+
ArrowBuf dataBuffer = imported.getDataBuffer();
727+
ByteBuffer nioBuffer = dataBuffer.nioBuffer().asReadOnlyBuffer();
728+
nioBuffer.order(ByteOrder.nativeOrder());
729+
assertEquals(1, nioBuffer.getInt(0));
730+
assertEquals(2, nioBuffer.getInt(1 << 2));
731+
assertEquals(3, nioBuffer.getInt(2 << 2));
732+
imported.close();
733+
}
734+
718735
@Test
719736
public void testImportReleasedArray() {
720737
// Consumer allocates empty structures

java/memory/memory-core/src/main/java/org/apache/arrow/memory/ArrowBuf.java

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -74,23 +74,23 @@ public final class ArrowBuf implements AutoCloseable {
7474
private long writerIndex;
7575
private final HistoricalLog historicalLog = BaseAllocator.DEBUG ?
7676
new HistoricalLog(BaseAllocator.DEBUG_LOG_LENGTH, "ArrowBuf[%d]", id) : null;
77-
private volatile long length;
77+
private volatile long capacity;
7878

7979
/**
8080
* Constructs a new ArrowBuf.
8181
*
8282
* @param referenceManager The memory manager to track memory usage and reference count of this buffer
83-
* @param length The byte length of this buffer
83+
* @param capacity The capacity in bytes of this buffer
8484
*/
8585
public ArrowBuf(
8686
final ReferenceManager referenceManager,
8787
final BufferManager bufferManager,
88-
final long length,
88+
final long capacity,
8989
final long memoryAddress) {
9090
this.referenceManager = referenceManager;
9191
this.bufferManager = bufferManager;
9292
this.addr = memoryAddress;
93-
this.length = length;
93+
this.capacity = capacity;
9494
this.readerIndex = 0;
9595
this.writerIndex = 0;
9696
if (BaseAllocator.DEBUG) {
@@ -136,7 +136,7 @@ public ReferenceManager getReferenceManager() {
136136
}
137137

138138
public long capacity() {
139-
return length;
139+
return capacity;
140140
}
141141

142142
/**
@@ -146,14 +146,14 @@ public long capacity() {
146146
*/
147147
public synchronized ArrowBuf capacity(long newCapacity) {
148148

149-
if (newCapacity == length) {
149+
if (newCapacity == capacity) {
150150
return this;
151151
}
152152

153153
Preconditions.checkArgument(newCapacity >= 0);
154154

155-
if (newCapacity < length) {
156-
length = newCapacity;
155+
if (newCapacity < capacity) {
156+
capacity = newCapacity;
157157
return this;
158158
}
159159

@@ -195,8 +195,8 @@ public ArrowBuf slice() {
195195
*/
196196
public ArrowBuf slice(long index, long length) {
197197

198-
Preconditions.checkPositionIndex(index, this.length);
199-
Preconditions.checkPositionIndex(index + length, this.length);
198+
Preconditions.checkPositionIndex(index, this.capacity);
199+
Preconditions.checkPositionIndex(index + length, this.capacity);
200200

201201
/*
202202
* Re the behavior of reference counting, see http://netty.io/wiki/reference-counted-objects
@@ -235,7 +235,7 @@ public long memoryAddress() {
235235

236236
@Override
237237
public String toString() {
238-
return String.format("ArrowBuf[%d], address:%d, length:%d", id, memoryAddress(), length);
238+
return String.format("ArrowBuf[%d], address:%d, capacity:%d", id, memoryAddress(), capacity);
239239
}
240240

241241
@Override

0 commit comments

Comments
 (0)