This is an automated email from the ASF dual-hosted git repository. samt pushed a commit to branch cassandra-3.11 in repository https://gitbox.apache.org/repos/asf/cassandra.git
commit d6beb0113f43233e4dc8b636328c3159ae61e969 Merge: 53740bc 092915a Author: Sam Tunnicliffe <[email protected]> AuthorDate: Thu Apr 16 18:28:33 2020 +0100 Merge branch 'cassandra-3.0' into cassandra-3.11 CHANGES.txt | 1 + .../cassandra/db/SinglePartitionReadCommand.java | 83 ++++++++--------- .../distributed/test/SimpleReadWriteTest.java | 101 +++++++++++++++++++++ 3 files changed, 142 insertions(+), 43 deletions(-) diff --cc CHANGES.txt index eb15f5f,91b8241..dbdb779 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,8 -1,5 +1,9 @@@ +3.11.7 + * Allow sstableloader to use SSL on the native port (CASSANDRA-14904) +Merged from 3.0: +======= 3.0.21 + * Don't skip sstables in slice queries based only on local min/max/deletion timestamp (CASSANDRA-15690) * Memtable memory allocations may deadlock (CASSANDRA-15367) * Run evictFromMembership in GossipStage (CASSANDRA-15592) Merged from 2.2: diff --cc src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java index bee4961,2e014ba..3d6559b --- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java +++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java @@@ -725,13 -710,13 +725,14 @@@ public class SinglePartitionReadComman * timestamp(tombstone) > maxTimestamp_s0 * since we necessarily have * timestamp(tombstone) <= maxTimestamp_s1 - * In other words, iterating in maxTimestamp order allow to do our mostRecentPartitionTombstone elimination - * in one pass, and minimize the number of sstables for which we read a partition tombstone. - */ + * In other words, iterating in descending maxTimestamp order allow to do our mostRecentPartitionTombstone + * elimination in one pass, and minimize the number of sstables for which we read a partition tombstone. - */ - Collections.sort(view.sstables, SSTableReader.maxTimestampComparator); ++ */ + Collections.sort(view.sstables, SSTableReader.maxTimestampDescending); long mostRecentPartitionTombstone = Long.MIN_VALUE; int nonIntersectingSSTables = 0; - List<SSTableReader> skippedSSTablesWithTombstones = null; + int includedDueToTombstones = 0; ++ SSTableReadMetricsCollector metricsCollector = new SSTableReadMetricsCollector(); for (SSTableReader sstable : view.sstables) @@@ -746,49 -731,60 +747,45 @@@ if (!sstable.isRepaired()) oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, sstable.getMinLocalDeletionTime()); + // 'iter' is added to iterators which is closed on exception, or through the closing of the final merged iterator + @SuppressWarnings("resource") - UnfilteredRowIterator iter = filter.filter(sstable.iterator(partitionKey(), - columnFilter(), - filter.isReversed(), - isForThrift(), - metricsCollector)); - - if (isForThrift()) - iter = ThriftResultsMerger.maybeWrap(iter, nowInSec()); - - iterators.add(RTBoundValidator.validate(iter, RTBoundValidator.Stage.SSTABLE, false)); ++ UnfilteredRowIterator iter = makeIterator(cfs, sstable, true, metricsCollector); + iterators.add(iter); - includedDueToTombstones++; + mostRecentPartitionTombstone = Math.max(mostRecentPartitionTombstone, + iter.partitionLevelDeletion().markedForDeleteAt()); + } + else + { + + nonIntersectingSSTables++; + // sstable contains no tombstone if maxLocalDeletionTime == Integer.MAX_VALUE, so we can safely skip those entirely - if (sstable.hasTombstones()) ++ if (sstable.mayHaveTombstones()) + { + // 'iter' is added to iterators which is closed on exception, or through the closing of the final merged iterator + @SuppressWarnings("resource") - UnfilteredRowIterator iter = filter.filter(sstable.iterator(partitionKey(), - columnFilter(), - filter.isReversed(), - isForThrift(), - metricsCollector)); ++ UnfilteredRowIterator iter = makeIterator(cfs, sstable, true, metricsCollector); + // if the sstable contains a partition delete, then we must include it regardless of whether it + // shadows any other data seen locally as we can't guarantee that other replicas have seen it + if (!iter.partitionLevelDeletion().isLive()) + { - includedDueToTombstones++; - iterators.add(RTBoundValidator.validate(iter, RTBoundValidator.Stage.SSTABLE, false)); + if (!sstable.isRepaired()) + oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, sstable.getMinLocalDeletionTime()); ++ iterators.add(iter); ++ includedDueToTombstones++; + mostRecentPartitionTombstone = Math.max(mostRecentPartitionTombstone, + iter.partitionLevelDeletion().markedForDeleteAt()); + } + else + { + iter.close(); + } - + } } } + if (Tracing.isTracing()) Tracing.trace("Skipped {}/{} non-slice-intersecting sstables, included {} due to tombstones", - nonIntersectingSSTables, view.sstables.size(), includedDueToTombstones); - - cfs.metric.updateSSTableIterated(metricsCollector.getMergedSSTables()); + nonIntersectingSSTables, view.sstables.size(), includedDueToTombstones); if (iterators.isEmpty()) return EmptyIterators.unfilteredRow(cfs.metadata, partitionKey(), filter.isReversed()); --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
