GitHub user vanzin opened a pull request:

    https://github.com/apache/spark/pull/9987

    [SPARK-12007] [network] Avoid copies in the network lib's RPC layer.

    This change seems large, but most of it is just replacing `byte[]`
    with `ByteBuffer` and `new byte[]` with `ByteBuffer.allocate()`,
    since it changes the network library's API.
    
    The following are parts of the code that actually have meaningful
    changes:
    
    - The Message implementations were changed to inherit from a new
      AbstractMessage that can optionally hold a reference to a body
      (in the form of a ManagedBuffer); this is similar to how
      ResponseWithBody worked before, except now it's not restricted
      to just responses.
    
    - The TransportFrameDecoder was pretty much rewritten to avoid
      copies as much as possible; it doesn't rely on CompositeByteBuf
      to accumulate incoming data anymore, since CompositeByteBuf
      has issues when slices are retained. The code now is able to
      create frames without having to resort to copying bytes except
      for a few bytes (containing the frame length) in very rare cases.
    
    - Some minor changes in the SASL layer to convert things back to
      `byte[]` since the JDK SASL API operates on those.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/vanzin/spark SPARK-12007

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/9987.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #9987
    
----
commit 83acf3794be1886bd8199d8a45f6ee12141db0e0
Author: Marcelo Vanzin <[email protected]>
Date:   2015-11-25T01:12:59Z

    [SPARK-12007] [network] Avoid copies in the network lib's RPC layer.
    
    This change seems large, but most of it is just replacing `byte[]`
    with `ByteBuffer` and `new byte[]` with `ByteBuffer.allocate()`,
    since it changes the network library's API.
    
    The following are parts of the code that actually have meaningful
    changes:
    
    - The Message implementations were changed to inherit from a new
      AbstractMessage that can optionally hold a reference to a body
      (in the form of a ManagedBuffer); this is similar to how
      ResponseWithBody worked before, except now it's not restricted
      to just responses.
    
    - The TransportFrameDecoder was pretty much rewritten to avoid
      copies as much as possible; it doesn't rely on CompositeByteBuf
      to accumulate incoming data anymore, since CompositeByteBuf
      has issues when slices are retained. The code now is able to
      create frames without having to resort to copying bytes except
      for a few bytes (containing the frame length) in very rare cases.
    
    - Some minor changes in the SASL layer to convert things back to
      `byte[]` since the JDK SASL API operates on those.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to