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;