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

ASF GitHub Bot commented on CASSANDRA-14556:
--------------------------------------------

Github user aweisberg commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/239#discussion_r202490979
  
    --- Diff: 
src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java ---
    @@ -249,4 +250,42 @@ public ByteBufAllocator getAllocator()
         {
             return channelConfig.getAllocator();
         }
    +
    +    /**
    +     * Consumes bytes in the stream until the given length
    +     *
    +     * @param writer
    +     * @param len
    +     * @return
    +     * @throws IOException
    +     */
    +    public long consumeUntil(BufferedDataOutputStreamPlus writer, long 
len) throws IOException
    +    {
    +        long copied = 0; // number of bytes copied
    +        while (copied < len)
    +        {
    +            if (buffer.remaining() == 0)
    +            {
    +                try
    +                {
    +                    reBuffer();
    +                }
    +                catch (EOFException e)
    +                {
    +                    throw new EOFException("EOF after " + copied + " bytes 
out of " + len);
    +                }
    +                if (buffer.remaining() == 0)
    +                    return copied == 0 ? -1 : copied;
    --- End diff --
    
    If reBuffer() fails it's going to throw an IOException or RuntimeException 
that won't be caught there so it won't throw AssertionError. It will only throw 
AssertionError when reBuffer() fails to reBuffer() and also doesn't exit via an 
Exception. 
    
    If reBuffer() doesn't reBuffer() and doesn't throw EOF then you don't know 
what's happening. That check doesn't do anything because if it doesn't read the 
require number of bytes it's going to throw EOF (in reBuffer()) and skip it 
anyways.


> Optimize streaming path in Cassandra
> ------------------------------------
>
>                 Key: CASSANDRA-14556
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Streaming and Messaging
>            Reporter: Dinesh Joshi
>            Assignee: Dinesh Joshi
>            Priority: Major
>              Labels: Performance
>             Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates 
> unnecessary garbage and slows down the whole streaming process as some 
> sstables can be transferred as a whole file rather than individual 
> partitions. The objective of the ticket is to detect when a whole sstable can 
> be transferred and skip the object reification. We can also use a zero-copy 
> path to avoid bringing data into user-space on both sending and receiving 
> side.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to