[ https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16553429#comment-16553429 ]
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_r204562307 --- 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 -- I mistakenly thought this index was a sampled index not a full index. Requiring a comparison of every partition key in every sstable for the entire data set seems like a big regression for some use cases. I was trying and failing to find the reasoning for why we switched to this. > 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