You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2020/04/01 20:59:38 UTC

[GitHub] [cassandra] tjake commented on a change in pull request #481: optimize ZCS file containment check by using file sections

tjake commented on a change in pull request #481: optimize ZCS file containment check by using file sections
URL: https://github.com/apache/cassandra/pull/481#discussion_r401903756
 
 

 ##########
 File path: src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java
 ##########
 @@ -185,46 +177,18 @@ public boolean shouldStreamEntireSSTable()
         if (!DatabaseDescriptor.streamEntireSSTables() || ref.get().getSSTableMetadata().hasLegacyCounterShards)
             return false;
 
-        ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(getTableId());
-
-        if (cfs == null)
-            return false;
-
-        AbstractCompactionStrategy compactionStrategy = cfs.getCompactionStrategyManager()
-                                                           .getCompactionStrategyFor(ref.get());
-
-        if (compactionStrategy instanceof LeveledCompactionStrategy)
-            return contained(normalizedRanges, ref.get());
-
-        return false;
+        return contained(sections, ref.get());
     }
 
     @VisibleForTesting
-    public boolean contained(List<Range<Token>> normalizedRanges, SSTableReader sstable)
+    public boolean contained(List<SSTableReader.PartitionPositionBounds> sections, SSTableReader sstable)
     {
-        if (isFullyContained != null)
-            return isFullyContained;
-
-        isFullyContained = computeContainment(normalizedRanges, sstable);
-        return isFullyContained;
-    }
-
-    private boolean computeContainment(List<Range<Token>> normalizedRanges, SSTableReader sstable)
-    {
-        if (normalizedRanges == null)
+        if (sections == null || sections.isEmpty())
             return false;
 
-        RangeOwnHelper rangeOwnHelper = new RangeOwnHelper(normalizedRanges);
-        try (KeyIterator iter = new KeyIterator(sstable.descriptor, sstable.metadata()))
-        {
-            while (iter.hasNext())
-            {
-                DecoratedKey key = iter.next();
-                if (!rangeOwnHelper.check(key))
-                    return false;
-            }
-        }
-        return true;
+        // if transfer sections contain entire sstable
+        long transferLength = sections.stream().mapToLong(p -> p.upperPosition - p.lowerPosition).sum();
 
 Review comment:
   These sections should be normalized, so you will only have 1 when its the full sstable right?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org