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

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

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

    https://github.com/apache/cassandra/pull/239#discussion_r204586299
  
    --- Diff: 
src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java ---
    @@ -114,13 +155,51 @@ public void write(StreamSession session, 
DataOutputStreamPlus out, int version)
             CassandraStreamHeader.serializer.serialize(header, out, version);
             out.flush();
     
    -        CassandraStreamWriter writer = header.compressionInfo == null ?
    -                                       new CassandraStreamWriter(sstable, 
header.sections, session) :
    -                                       new 
CompressedCassandraStreamWriter(sstable, header.sections,
    -                                                                           
header.compressionInfo, session);
    +        IStreamWriter writer;
    +        if (shouldStreamFullSSTable())
    +        {
    +            writer = new CassandraBlockStreamWriter(sstable, session, 
components);
    +        }
    +        else
    +        {
    +            writer = (header.compressionInfo == null) ?
    +                     new CassandraStreamWriter(sstable, header.sections, 
session) :
    +                     new CompressedCassandraStreamWriter(sstable, 
header.sections,
    +                                                         
header.compressionInfo, session);
    +        }
             writer.write(out);
         }
     
    +    @VisibleForTesting
    +    public boolean shouldStreamFullSSTable()
    +    {
    +        return isFullSSTableTransfersEnabled && isFullyContained;
    +    }
    +
    +    @VisibleForTesting
    +    public boolean fullyContainedIn(List<Range<Token>> normalizedRanges, 
SSTableReader sstable)
    +    {
    +        if (normalizedRanges == null)
    +            return false;
    +
    +        RangeOwnHelper rangeOwnHelper = new 
RangeOwnHelper(normalizedRanges);
    +        try (KeyIterator iter = new KeyIterator(sstable.descriptor, 
sstable.metadata()))
    +        {
    +            while (iter.hasNext())
    +            {
    +                DecoratedKey key = iter.next();
    +                try
    +                {
    +                    rangeOwnHelper.check(key);
    +                } catch(RuntimeException e)
    --- End diff --
    
    @iamaleksey thank you for the useful feedback. I did discuss this with 
@krummas and I believe while there was a room for improvement, the thinking 
back then was that the benefits would outweigh the cost. I looked through the 
codebase and this was the best way to definitely verify range containment as I 
was going for correctness. That said, what you suggest is obviously better. I 
am concerned about scope creep in this PR. Would it be ok if we address it as 
part of a separate PR?
    
    It would also be useful, if we could design the effective range computation 
and storage in the metadata. I am not sure what sort of gotchas I might run 
into.


> 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