You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/08/11 09:59:00 UTC

[jira] [Commented] (FLINK-7428) avoid one additional buffer copy when receiving messages

    [ https://issues.apache.org/jira/browse/FLINK-7428?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16123115#comment-16123115 ] 

ASF GitHub Bot commented on FLINK-7428:
---------------------------------------

GitHub user NicoK opened a pull request:

    https://github.com/apache/flink/pull/4529

    [FLINK-7428][network] avoid buffer copies when receiving messages

    ## What is the purpose of the change
    
    The `LengthFieldBasedFrameDecoder` used so far creates one additional copy in its `#extractFrame()` method which is avoidable in our use case and changed by this PR (based upon #4528).
    
    ## Brief change log
    
    - let `NettyMessageDecoder` inherit from `LengthFieldBasedFrameDecoder` (instead of being an additional step in the pipeline)
    - override `LengthFieldBasedFrameDecoder#extractFrame()` similarly to the `ObjectDecoder` class provided by netty itself
    
    ## Verifying this change
    
    This change is already covered by existing tests, such as NettyMessageSerializationTest as well as many other tests involving network communication, e.g. IT cases.
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (no)
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
      - The serializers: (yes: network "deserialisation")
      - The runtime per-record code paths (performance sensitive): (yes)
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes: network communication)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (no)
      - If yes, how is the feature documented? (JavaDocs)
    


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

    $ git pull https://github.com/NicoK/flink flink-7428

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

    https://github.com/apache/flink/pull/4529.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 #4529
    
----
commit 1c0e5aae05f3e4eaec986792b508a08ee242ac73
Author: Nico Kruber <ni...@data-artisans.com>
Date:   2017-08-10T14:58:19Z

    [FLINK-7427][network] integrate PartitionRequestProtocol into NettyProtocol
    
    - removes one level of (unneeded) abstraction for clarity

commit 18c665a2bec35b98da165f243b64faaf4d6ea056
Author: Nico Kruber <ni...@data-artisans.com>
Date:   2017-08-10T16:23:18Z

    [FLINK-7428][network] avoid buffer copies when receiving messages
    
    The LengthFieldBasedFrameDecoder used so far creates one additional copy in its
    `#extractFrame()` method which is avoidable in our case. We do so by letting
    `NettyMessageDecoder` inherit from it (instead of being an additional step in
    the pipeline) and overriding this similarly to the `ObjectDecoder` class
    provided by netty itself.

----


> avoid one additional buffer copy when receiving messages
> --------------------------------------------------------
>
>                 Key: FLINK-7428
>                 URL: https://issues.apache.org/jira/browse/FLINK-7428
>             Project: Flink
>          Issue Type: Improvement
>          Components: Network
>    Affects Versions: 1.4.0
>            Reporter: Nico Kruber
>            Assignee: Nico Kruber
>
> By using {{LengthFieldBasedFrameDecoder}}, we create one unnecessary (netty) buffer copy in this class which could be easily avoided since we can ensure that the buffer is free to be released after decoding it in the {{NettyMessageDecoder}} and into our own buffer and/or events.
> The solution would be to make {{NettyMessageDecoder}} extend from {{LengthFieldBasedFrameDecoder}} and handle the decoding of the frames and the objects in there. In the frame creation otherwise done by {{LengthFieldBasedFrameDecoder}}, we could use a sliced buffer instead. This solution also makes the channel pipelines a bit simpler.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)