You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by li...@apache.org on 2021/02/03 23:03:50 UTC
[arrow] branch master updated: ARROW-11066: [Java][FlightRPC] fix
zero-copy optimization
This is an automated email from the ASF dual-hosted git repository.
lidavidm pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git
The following commit(s) were added to refs/heads/master by this push:
new 5e5c2b4 ARROW-11066: [Java][FlightRPC] fix zero-copy optimization
5e5c2b4 is described below
commit 5e5c2b4890fd267b7d0d2734977e735ae78adcc9
Author: David Li <li...@gmail.com>
AuthorDate: Wed Feb 3 18:02:24 2021 -0500
ARROW-11066: [Java][FlightRPC] fix zero-copy optimization
Flight's zero-copy optimization was not enabled previously due to an oversight. Additionally, enabling it exposed a use-after-free of a direct Netty ByteBuf that manifested as SIGSEGV. This fixes both issues: the fast path is truly enabled, and to avoid the use-after-free, the lifetimes of Netty ByteBufs in the Flight gRPC marshaller is now tied to the ArrowBuf lifetimes. Finally, an environment variable + property are added to disable the fast path in case the old behavior is desirable.
Closes #9387 from lidavidm/arrow-11066
Authored-by: David Li <li...@gmail.com>
Signed-off-by: David Li <li...@gmail.com>
---
.../java/org/apache/arrow/flight/ArrowMessage.java | 76 +++++++++++++++++++---
.../arrow/flight/grpc/AddWritableBuffer.java | 14 ++--
.../apache/arrow/flight/TestBasicOperation.java | 5 ++
.../arrow/flight/perf/PerformanceTestServer.java | 2 +-
4 files changed, 81 insertions(+), 16 deletions(-)
diff --git a/java/flight/flight-core/src/main/java/org/apache/arrow/flight/ArrowMessage.java b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/ArrowMessage.java
index 06d3bd3..ce875d1 100644
--- a/java/flight/flight-core/src/main/java/org/apache/arrow/flight/ArrowMessage.java
+++ b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/ArrowMessage.java
@@ -67,7 +67,17 @@ import io.netty.buffer.UnpooledByteBufAllocator;
*/
class ArrowMessage implements AutoCloseable {
- public static final boolean FAST_PATH = true;
+ // If true, serialize Arrow data by giving gRPC a reference to the underlying Arrow buffer
+ // instead of copying the data. Defaults to true.
+ public static final boolean ENABLE_ZERO_COPY;
+
+ static {
+ String zeroCopyFlag = System.getProperty("arrow.flight.enable_zero_copy");
+ if (zeroCopyFlag == null) {
+ zeroCopyFlag = System.getenv("ARROW_FLIGHT_ENABLE_ZERO_COPY");
+ }
+ ENABLE_ZERO_COPY = !"false".equals(zeroCopyFlag);
+ }
private static final int DESCRIPTOR_TAG =
(FlightData.FLIGHT_DESCRIPTOR_FIELD_NUMBER << 3) | WireFormat.WIRETYPE_LENGTH_DELIMITED;
@@ -279,7 +289,7 @@ class ArrowMessage implements AutoCloseable {
case APP_METADATA_TAG: {
int size = readRawVarint32(stream);
appMetadata = allocator.buffer(size);
- GetReadableBuffer.readIntoBuffer(stream, appMetadata, size, FAST_PATH);
+ GetReadableBuffer.readIntoBuffer(stream, appMetadata, size, ENABLE_ZERO_COPY);
break;
}
case BODY_TAG:
@@ -290,7 +300,7 @@ class ArrowMessage implements AutoCloseable {
}
int size = readRawVarint32(stream);
body = allocator.buffer(size);
- GetReadableBuffer.readIntoBuffer(stream, body, size, FAST_PATH);
+ GetReadableBuffer.readIntoBuffer(stream, body, size, ENABLE_ZERO_COPY);
break;
default:
@@ -392,6 +402,9 @@ class ArrowMessage implements AutoCloseable {
int size = 0;
List<ByteBuf> allBufs = new ArrayList<>();
for (ArrowBuf b : bufs) {
+ // [ARROW-11066] This creates a Netty buffer whose refcnt is INDEPENDENT of the backing
+ // Arrow buffer. This is susceptible to use-after-free, so we subclass CompositeByteBuf
+ // below to tie the Arrow buffer refcnt to the Netty buffer refcnt
allBufs.add(Unpooled.wrappedBuffer(b.nioBuffer()).retain());
size += b.readableBytes();
// [ARROW-4213] These buffers must be aligned to an 8-byte boundary in order to be readable from C++.
@@ -408,12 +421,13 @@ class ArrowMessage implements AutoCloseable {
ByteBuf initialBuf = Unpooled.buffer(baos.size());
initialBuf.writeBytes(baos.toByteArray());
- final CompositeByteBuf bb = new CompositeByteBuf(UnpooledByteBufAllocator.DEFAULT, true,
+ final CompositeByteBuf bb = new ArrowBufRetainingCompositeByteBuf(
Math.max(2, bufs.size() + 1),
ImmutableList.<ByteBuf>builder()
.add(initialBuf)
.addAll(allBufs)
- .build());
+ .build(),
+ bufs);
final ByteBufInputStream is = new DrainableByteBufInputStream(bb);
return is;
} catch (Exception ex) {
@@ -422,6 +436,52 @@ class ArrowMessage implements AutoCloseable {
}
+ /**
+ * ARROW-11066: enable the zero-copy optimization and protect against use-after-free.
+ *
+ * When you send a message through gRPC, the following happens:
+ * 1. gRPC immediately serializes the message, eventually calling asInputStream above.
+ * 2. gRPC buffers the serialized message for sending.
+ * 3. Later, gRPC will actually write out the message.
+ *
+ * The problem with this is that when the zero-copy optimization is enabled, Flight
+ * "serializes" the message by handing gRPC references to Arrow data. That means we need
+ * a way to keep the Arrow buffers valid until gRPC actually writes them, else, we'll read
+ * invalid data or segfault. gRPC doesn't know anything about Arrow buffers, either.
+ *
+ * This class solves that issue by bridging Arrow and Netty/gRPC. We increment the refcnt
+ * on a set of Arrow backing buffers and decrement them once the Netty buffers are freed
+ * by gRPC.
+ */
+ private static final class ArrowBufRetainingCompositeByteBuf extends CompositeByteBuf {
+ // Arrow buffers that back the Netty ByteBufs here; ByteBufs held by this class are
+ // either slices of one of the ArrowBufs or independently allocated.
+ final List<ArrowBuf> backingBuffers;
+ boolean freed;
+
+ ArrowBufRetainingCompositeByteBuf(int maxNumComponents, Iterable<ByteBuf> buffers, List<ArrowBuf> backingBuffers) {
+ super(UnpooledByteBufAllocator.DEFAULT, /* direct */ true, maxNumComponents, buffers);
+ this.backingBuffers = backingBuffers;
+ this.freed = false;
+ // N.B. the Netty superclass avoids enhanced-for to reduce GC pressure, so follow that here
+ for (int i = 0; i < backingBuffers.size(); i++) {
+ backingBuffers.get(i).getReferenceManager().retain();
+ }
+ }
+
+ @Override
+ protected void deallocate() {
+ super.deallocate();
+ if (freed) {
+ return;
+ }
+ freed = true;
+ for (int i = 0; i < backingBuffers.size(); i++) {
+ backingBuffers.get(i).getReferenceManager().release();
+ }
+ }
+ }
+
private static class DrainableByteBufInputStream extends ByteBufInputStream implements Drainable {
private final CompositeByteBuf buf;
@@ -434,11 +494,7 @@ class ArrowMessage implements AutoCloseable {
@Override
public int drainTo(OutputStream target) throws IOException {
int size = buf.readableBytes();
- if (FAST_PATH && AddWritableBuffer.add(buf, target)) {
- return size;
- }
-
- buf.getBytes(0, target, buf.readableBytes());
+ AddWritableBuffer.add(buf, target, ENABLE_ZERO_COPY);
return size;
}
diff --git a/java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/AddWritableBuffer.java b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/AddWritableBuffer.java
index 07b19c2..6b66bb6 100644
--- a/java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/AddWritableBuffer.java
+++ b/java/flight/flight-core/src/main/java/org/apache/arrow/flight/grpc/AddWritableBuffer.java
@@ -84,14 +84,18 @@ public class AddWritableBuffer {
}
/**
- * Add the provided ByteBuf to the output stream if it is possible.
+ * Add the provided ByteBuf to the gRPC BufferChainOutputStream if possible, else copy the buffer to the stream.
* @param buf The buffer to add.
* @param stream The Candidate OutputStream to add to.
- * @return True if added. False if not possible.
- * @throws IOException on error
+ * @param tryZeroCopy If true, try to zero-copy append the buffer to the stream. This may not succeed.
+ * @return True if buffer was zero-copy added to the stream. False if the buffer was copied.
+ * @throws IOException if the fast path is not enabled and there was an error copying the buffer to the stream.
*/
- public static boolean add(ByteBuf buf, OutputStream stream) throws IOException {
- buf.readBytes(stream, buf.readableBytes());
+ public static boolean add(ByteBuf buf, OutputStream stream, boolean tryZeroCopy) throws IOException {
+ if (!tryZeroCopy) {
+ buf.getBytes(0, stream, buf.readableBytes());
+ return false;
+ }
if (bufChainOut == null) {
return false;
diff --git a/java/flight/flight-core/src/test/java/org/apache/arrow/flight/TestBasicOperation.java b/java/flight/flight-core/src/test/java/org/apache/arrow/flight/TestBasicOperation.java
index daf911d..4db94cc 100644
--- a/java/flight/flight-core/src/test/java/org/apache/arrow/flight/TestBasicOperation.java
+++ b/java/flight/flight-core/src/test/java/org/apache/arrow/flight/TestBasicOperation.java
@@ -63,6 +63,11 @@ import io.grpc.MethodDescriptor;
*/
public class TestBasicOperation {
+ @Test
+ public void fastPathEnabledByDefault() {
+ Assert.assertTrue(ArrowMessage.ENABLE_ZERO_COPY);
+ }
+
/**
* ARROW-6017: we should be able to construct locations for unknown schemes.
*/
diff --git a/java/flight/flight-core/src/test/java/org/apache/arrow/flight/perf/PerformanceTestServer.java b/java/flight/flight-core/src/test/java/org/apache/arrow/flight/perf/PerformanceTestServer.java
index 4200bbe..a797e55 100644
--- a/java/flight/flight-core/src/test/java/org/apache/arrow/flight/perf/PerformanceTestServer.java
+++ b/java/flight/flight-core/src/test/java/org/apache/arrow/flight/perf/PerformanceTestServer.java
@@ -68,7 +68,7 @@ public class PerformanceTestServer implements AutoCloseable {
@Override
public WaitResult waitForListener(long timeout) {
- while (!listener.isReady()) {
+ while (!listener.isReady() && !listener.isCancelled()) {
// busy wait
}
return WaitResult.READY;