Merge branch 'cassandra-3.0' into cassandra-3.3

Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f2174280
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f2174280
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f2174280

Branch: refs/heads/trunk
Commit: f2174280d6dcbb951b539dc558d1e1ae81f43b5c
Parents: acb7fed 442f473
Author: Marcus Eriksson <marc...@apache.org>
Authored: Mon Jan 25 10:09:15 2016 +0100
Committer: Marcus Eriksson <marc...@apache.org>
Committed: Mon Jan 25 10:09:15 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 src/java/org/apache/cassandra/db/Memtable.java  |   9 +
 .../db/compaction/CompactionController.java     |  19 +-
 .../db/compaction/CompactionControllerTest.java | 195 +++++++++++++++++++
 4 files changed, 221 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2174280/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index ed56a78,70abffe..81657f9
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -15,9 -11,20 +15,10 @@@ Merged from 3.0
     tombstone (CASSANDRA-10743)
   * MV should use the maximum timestamp of the primary key (CASSANDRA-10910)
   * Fix potential assertion error during compaction (CASSANDRA-10944)
 - * Fix counting of received sstables in streaming (CASSANDRA-10949)
 - * Implement hints compression (CASSANDRA-9428)
 - * Fix potential assertion error when reading static columns (CASSANDRA-10903)
 - * Avoid NoSuchElementException when executing empty batch (CASSANDRA-10711)
 - * Avoid building PartitionUpdate in toString (CASSANDRA-10897)
 - * Reduce heap spent when receiving many SSTables (CASSANDRA-10797)
 - * Add back support for 3rd party auth providers to bulk loader 
(CASSANDRA-10873)
 - * Eliminate the dependency on jgrapht for UDT resolution (CASSANDRA-10653)
 - * (Hadoop) Close Clusters and Sessions in Hadoop Input/Output classes 
(CASSANDRA-10837)
 - * Fix sstableloader not working with upper case keyspace name 
(CASSANDRA-10806)
  Merged from 2.2:
 -2.2.5
+  * maxPurgeableTimestamp needs to check memtables too (CASSANDRA-9949)
   * Apply change to compaction throughput in real time (CASSANDRA-10025)
 + * (cqlsh) encode input correctly when saving history
   * Fix potential NPE on ORDER BY queries with IN (CASSANDRA-10955)
   * Start L0 STCS-compactions even if there is a L0 -> L1 compaction
     going (CASSANDRA-10979)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2174280/src/java/org/apache/cassandra/db/Memtable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Memtable.java
index 8e7a43c,5d5f7bf..952c045
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@@ -334,37 -316,24 +338,42 @@@ public class Memtable implements Compar
          return creationTime;
      }
  
+     public long getMinTimestamp()
+     {
+         return minTimestamp;
+     }
+ 
 -    class FlushRunnable extends DiskAwareRunnable
 +    class FlushRunnable implements Callable<SSTableMultiWriter>
      {
 -        private final ReplayPosition context;
 +        public final ReplayPosition context;
          private final long estimatedSize;
 +        private final ConcurrentNavigableMap<PartitionPosition, 
AtomicBTreePartition> toFlush;
  
          private final boolean isBatchLogTable;
 +        private final SSTableMultiWriter writer;
 +
 +        // keeping these to be able to log what we are actually flushing
 +        private final PartitionPosition from;
 +        private final PartitionPosition to;
  
 -        FlushRunnable(ReplayPosition context)
 +        FlushRunnable(ReplayPosition context, PartitionPosition from, 
PartitionPosition to, Directories.DataDirectory flushLocation, 
LifecycleTransaction txn)
          {
 -            this.context = context;
 +            this(context, partitions.subMap(from, to), flushLocation, from, 
to, txn);
 +        }
 +
 +        FlushRunnable(ReplayPosition context, LifecycleTransaction txn)
 +        {
 +            this(context, partitions, null, null, null, txn);
 +        }
  
 +        FlushRunnable(ReplayPosition context, 
ConcurrentNavigableMap<PartitionPosition, AtomicBTreePartition> toFlush, 
Directories.DataDirectory flushLocation, PartitionPosition from, 
PartitionPosition to, LifecycleTransaction txn)
 +        {
 +            this.context = context;
 +            this.toFlush = toFlush;
 +            this.from = from;
 +            this.to = to;
              long keySize = 0;
 -            for (PartitionPosition key : partitions.keySet())
 +            for (PartitionPosition key : toFlush.keySet())
              {
                  //  make sure we don't write non-sensical keys
                  assert key instanceof DecoratedKey;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2174280/src/java/org/apache/cassandra/db/compaction/CompactionController.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionController.java
index a7224a1,5cb60c5..79c6d53
--- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
@@@ -190,10 -196,18 +196,17 @@@ public class CompactionController imple
          {
              // if we don't have bloom filter(bf_fp_chance=1.0 or filter file 
is missing),
              // we check index file instead.
 -            if (sstable.getBloomFilter() instanceof AlwaysPresentFilter && 
sstable.getPosition(key, SSTableReader.Operator.EQ, false) != null)
 -                min = Math.min(min, sstable.getMinTimestamp());
 -            else if (sstable.getBloomFilter().isPresent(key))
 +            if ((sstable.getBloomFilter() instanceof AlwaysPresentFilter && 
sstable.getPosition(key, SSTableReader.Operator.EQ, false) != null)
 +                || sstable.getBloomFilter().isPresent(key))
                  min = Math.min(min, sstable.getMinTimestamp());
          }
+ 
+         for (Memtable memtable : cfs.getTracker().getView().getAllMemtables())
+         {
+             Partition partition = memtable.getPartition(key);
+             if (partition != null)
+                 min = Math.min(min, partition.stats().minTimestamp);
+         }
          return min;
      }
  

Reply via email to