You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2015/11/20 11:10:17 UTC

[1/5] cassandra git commit: Limit size of windows with DTCS

Repository: cassandra
Updated Branches:
  refs/heads/trunk f81a91d3f -> 0caa17de0


Limit size of windows with DTCS

Patch by marcuse; reviewed by Branimir Lambov for CASSANDRA-10280


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

Branch: refs/heads/trunk
Commit: 99617a529378f00cb86ab733959c7be9966860c9
Parents: 4f2337f
Author: Marcus Eriksson <ma...@apache.org>
Authored: Tue Sep 8 13:50:16 2015 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Fri Nov 20 10:59:46 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 pylib/cqlshlib/cql3handling.py                  |  1 +
 .../DateTieredCompactionStrategy.java           | 33 +++++++++--------
 .../DateTieredCompactionStrategyOptions.java    | 30 ++++++++++++++--
 .../DateTieredCompactionStrategyTest.java       | 38 +++++++++++++-------
 5 files changed, 73 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/99617a52/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 94a9ae2..66423c7 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.12
+ * Limit window size in DTCS (CASSANDRA-10280)
  * sstableloader does not use MAX_HEAP_SIZE env parameter (CASSANDRA-10188)
  * (cqlsh) Improve COPY TO performance and error handling (CASSANDRA-9304)
  * Don't remove level info when running upgradesstables (CASSANDRA-10692)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/99617a52/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 49970e4..38f118f 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -472,6 +472,7 @@ def cf_prop_val_mapkey_completer(ctxt, cass):
             opts.add('max_sstable_age_days')
             opts.add('timestamp_resolution')
             opts.add('min_threshold')
+            opts.add('max_window_size_seconds')
         return map(escape_value, opts)
     return ()
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/99617a52/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
index ece596f..ae684ec 100644
--- a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
@@ -131,7 +131,7 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
     {
         Iterable<SSTableReader> candidates = filterOldSSTables(Lists.newArrayList(candidateSSTables), options.maxSSTableAge, now);
 
-        List<List<SSTableReader>> buckets = getBuckets(createSSTableAndMinTimestampPairs(candidates), options.baseTime, base, now);
+        List<List<SSTableReader>> buckets = getBuckets(createSSTableAndMinTimestampPairs(candidates), options.baseTime, base, now, options.maxWindowSize);
         logger.debug("Compaction buckets are {}", buckets);
         updateEstimatedCompactionsByTasks(buckets);
         List<SSTableReader> mostInteresting = newestBucket(buckets,
@@ -139,6 +139,7 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
                                                            cfs.getMaximumCompactionThreshold(),
                                                            now,
                                                            options.baseTime,
+                                                           options.maxWindowSize,
                                                            stcsOptions);
         if (!mostInteresting.isEmpty())
             return mostInteresting;
@@ -217,10 +218,13 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
         // A timestamp t hits the target iff t / size == divPosition.
         public final long divPosition;
 
-        public Target(long size, long divPosition)
+        public final long maxWindowSize;
+
+        public Target(long size, long divPosition, long maxWindowSize)
         {
             this.size = size;
             this.divPosition = divPosition;
+            this.maxWindowSize = maxWindowSize;
         }
 
         /**
@@ -250,10 +254,10 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
          */
         public Target nextTarget(int base)
         {
-            if (divPosition % base > 0)
-                return new Target(size, divPosition - 1);
+            if (divPosition % base > 0 || size * base > maxWindowSize)
+                return new Target(size, divPosition - 1, maxWindowSize);
             else
-                return new Target(size * base, divPosition / base - 1);
+                return new Target(size * base, divPosition / base - 1, maxWindowSize);
         }
     }
 
@@ -270,7 +274,7 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
      *         Each bucket is also a list of files ordered from newest to oldest.
      */
     @VisibleForTesting
-    static <T> List<List<T>> getBuckets(Collection<Pair<T, Long>> files, long timeUnit, int base, long now)
+    static <T> List<List<T>> getBuckets(Collection<Pair<T, Long>> files, long timeUnit, int base, long now, long maxWindowSize)
     {
         // Sort files by age. Newest first.
         final List<Pair<T, Long>> sortedFiles = Lists.newArrayList(files);
@@ -283,7 +287,7 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
         }));
 
         List<List<T>> buckets = Lists.newArrayList();
-        Target target = getInitialTarget(now, timeUnit);
+        Target target = getInitialTarget(now, timeUnit, maxWindowSize);
         PeekingIterator<Pair<T, Long>> it = Iterators.peekingIterator(sortedFiles.iterator());
 
         outerLoop:
@@ -302,7 +306,6 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
                 else // If the file is too old for the target, switch targets.
                     target = target.nextTarget(base);
             }
-
             List<T> bucket = Lists.newArrayList();
             while (target.onTarget(it.peek().right))
             {
@@ -318,9 +321,9 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
     }
 
     @VisibleForTesting
-    static Target getInitialTarget(long now, long timeUnit)
+    static Target getInitialTarget(long now, long timeUnit, long maxWindowSize)
     {
-        return new Target(timeUnit, now / timeUnit);
+        return new Target(timeUnit, now / timeUnit, maxWindowSize);
     }
 
 
@@ -329,8 +332,9 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
         int n = 0;
         for (List<SSTableReader> bucket : tasks)
         {
-            if (bucket.size() >= cfs.getMinimumCompactionThreshold())
-                n += getSTCSBuckets(bucket, stcsOptions).size();
+            for (List<SSTableReader> stcsBucket : getSTCSBuckets(bucket, stcsOptions))
+                if (stcsBucket.size() >= cfs.getMinimumCompactionThreshold())
+                    n += Math.ceil((double)stcsBucket.size() / cfs.getMaximumCompactionThreshold());
         }
         estimatedRemainingTasks = n;
     }
@@ -343,12 +347,12 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
      * @return a bucket (list) of sstables to compact.
      */
     @VisibleForTesting
-    static List<SSTableReader> newestBucket(List<List<SSTableReader>> buckets, int minThreshold, int maxThreshold, long now, long baseTime, SizeTieredCompactionStrategyOptions stcsOptions)
+    static List<SSTableReader> newestBucket(List<List<SSTableReader>> buckets, int minThreshold, int maxThreshold, long now, long baseTime, long maxWindowSize, SizeTieredCompactionStrategyOptions stcsOptions)
     {
         // If the "incoming window" has at least minThreshold SSTables, choose that one.
         // For any other bucket, at least 2 SSTables is enough.
         // In any case, limit to maxThreshold SSTables.
-        Target incomingWindow = getInitialTarget(now, baseTime);
+        Target incomingWindow = getInitialTarget(now, baseTime, maxWindowSize);
         for (List<SSTableReader> bucket : buckets)
         {
             boolean inFirstWindow = incomingWindow.onTarget(bucket.get(0).getMinTimestamp());
@@ -412,7 +416,6 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
         return Long.MAX_VALUE;
     }
 
-
     public static Map<String, String> validateOptions(Map<String, String> options) throws ConfigurationException
     {
         Map<String, String> uncheckedOptions = AbstractCompactionStrategy.validateOptions(options);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/99617a52/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyOptions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyOptions.java b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyOptions.java
index 0cbf90e..5803115 100644
--- a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyOptions.java
+++ b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyOptions.java
@@ -25,17 +25,24 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 public final class DateTieredCompactionStrategyOptions
 {
     protected static final TimeUnit DEFAULT_TIMESTAMP_RESOLUTION = TimeUnit.MICROSECONDS;
-    protected static final double DEFAULT_MAX_SSTABLE_AGE_DAYS = 365;
+    @Deprecated
+    protected static final double DEFAULT_MAX_SSTABLE_AGE_DAYS = 365*1000;
     protected static final long DEFAULT_BASE_TIME_SECONDS = 60;
+    protected static final long DEFAULT_MAX_WINDOW_SIZE_SECONDS = TimeUnit.SECONDS.convert(1, TimeUnit.DAYS);
+
     protected static final int DEFAULT_EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS = 60 * 10;
     protected static final String TIMESTAMP_RESOLUTION_KEY = "timestamp_resolution";
+    @Deprecated
     protected static final String MAX_SSTABLE_AGE_KEY = "max_sstable_age_days";
     protected static final String BASE_TIME_KEY = "base_time_seconds";
     protected static final String EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS_KEY = "expired_sstable_check_frequency_seconds";
+    protected static final String MAX_WINDOW_SIZE_KEY = "max_window_size_seconds";
 
+    @Deprecated
     protected final long maxSSTableAge;
     protected final long baseTime;
     protected final long expiredSSTableCheckFrequency;
+    protected final long maxWindowSize;
 
     public DateTieredCompactionStrategyOptions(Map<String, String> options)
     {
@@ -48,13 +55,16 @@ public final class DateTieredCompactionStrategyOptions
         baseTime = timestampResolution.convert(optionValue == null ? DEFAULT_BASE_TIME_SECONDS : Long.parseLong(optionValue), TimeUnit.SECONDS);
         optionValue = options.get(EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS_KEY);
         expiredSSTableCheckFrequency = TimeUnit.MILLISECONDS.convert(optionValue == null ? DEFAULT_EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS : Long.parseLong(optionValue), TimeUnit.SECONDS);
+        optionValue = options.get(MAX_WINDOW_SIZE_KEY);
+        maxWindowSize = timestampResolution.convert(optionValue == null ? DEFAULT_MAX_WINDOW_SIZE_SECONDS : Long.parseLong(optionValue), TimeUnit.SECONDS);
     }
 
     public DateTieredCompactionStrategyOptions()
     {
-        maxSSTableAge = Math.round(DEFAULT_MAX_SSTABLE_AGE_DAYS * DEFAULT_TIMESTAMP_RESOLUTION.convert(1, TimeUnit.DAYS));
+        maxSSTableAge = Math.round(DEFAULT_MAX_SSTABLE_AGE_DAYS * DEFAULT_TIMESTAMP_RESOLUTION.convert((long) DEFAULT_MAX_SSTABLE_AGE_DAYS, TimeUnit.DAYS));
         baseTime = DEFAULT_TIMESTAMP_RESOLUTION.convert(DEFAULT_BASE_TIME_SECONDS, TimeUnit.SECONDS);
         expiredSSTableCheckFrequency = TimeUnit.MILLISECONDS.convert(DEFAULT_EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS, TimeUnit.SECONDS);
+        maxWindowSize = DEFAULT_TIMESTAMP_RESOLUTION.convert(1, TimeUnit.DAYS);
     }
 
     public static Map<String, String> validateOptions(Map<String, String> options, Map<String, String> uncheckedOptions) throws  ConfigurationException
@@ -112,10 +122,26 @@ public final class DateTieredCompactionStrategyOptions
             throw new ConfigurationException(String.format("%s is not a parsable int (base10) for %s", optionValue, EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS_KEY), e);
         }
 
+        optionValue = options.get(MAX_WINDOW_SIZE_KEY);
+        try
+        {
+            long maxWindowSize = optionValue == null ? DEFAULT_MAX_WINDOW_SIZE_SECONDS : Long.parseLong(optionValue);
+            if (maxWindowSize < 0)
+            {
+                throw new ConfigurationException(String.format("%s must not be negative, but was %d", MAX_WINDOW_SIZE_KEY, maxWindowSize));
+            }
+        }
+        catch (NumberFormatException e)
+        {
+            throw new ConfigurationException(String.format("%s is not a parsable int (base10) for %s", optionValue, MAX_WINDOW_SIZE_KEY), e);
+        }
+
+
         uncheckedOptions.remove(MAX_SSTABLE_AGE_KEY);
         uncheckedOptions.remove(BASE_TIME_KEY);
         uncheckedOptions.remove(TIMESTAMP_RESOLUTION_KEY);
         uncheckedOptions.remove(EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS_KEY);
+        uncheckedOptions.remove(MAX_WINDOW_SIZE_KEY);
 
         return uncheckedOptions;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/99617a52/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
index 368101d..5afd575 100644
--- a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
@@ -88,6 +88,17 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
             options.put(DateTieredCompactionStrategyOptions.MAX_SSTABLE_AGE_KEY, "0");
         }
 
+        try
+        {
+            options.put(DateTieredCompactionStrategyOptions.MAX_WINDOW_SIZE_KEY, "-1");
+            validateOptions(options);
+            fail(String.format("Negative %s should be rejected", DateTieredCompactionStrategyOptions.MAX_WINDOW_SIZE_KEY));
+        }
+        catch (ConfigurationException e)
+        {
+            options.put(DateTieredCompactionStrategyOptions.MAX_WINDOW_SIZE_KEY, "0");
+        }
+
         options.put("bad_option", "1.0");
         unvalidated = validateOptions(options);
         assertTrue(unvalidated.containsKey("bad_option"));
@@ -101,11 +112,11 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
         options.put(DateTieredCompactionStrategyOptions.TIMESTAMP_RESOLUTION_KEY, "SECONDS");
 
         DateTieredCompactionStrategyOptions opts = new DateTieredCompactionStrategyOptions(options);
-        assertEquals(opts.maxSSTableAge, TimeUnit.SECONDS.convert(365, TimeUnit.DAYS));
+        assertEquals(opts.maxSSTableAge, TimeUnit.SECONDS.convert(365*1000, TimeUnit.DAYS));
 
         options.put(DateTieredCompactionStrategyOptions.TIMESTAMP_RESOLUTION_KEY, "MILLISECONDS");
         opts = new DateTieredCompactionStrategyOptions(options);
-        assertEquals(opts.maxSSTableAge, TimeUnit.MILLISECONDS.convert(365, TimeUnit.DAYS));
+        assertEquals(opts.maxSSTableAge, TimeUnit.MILLISECONDS.convert(365*1000, TimeUnit.DAYS));
 
         options.put(DateTieredCompactionStrategyOptions.TIMESTAMP_RESOLUTION_KEY, "MICROSECONDS");
         options.put(DateTieredCompactionStrategyOptions.MAX_SSTABLE_AGE_KEY, "10");
@@ -132,7 +143,7 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
                 Pair.create("a", 1L),
                 Pair.create("b", 201L)
         );
-        List<List<String>> buckets = getBuckets(pairs, 100L, 2, 200L);
+        List<List<String>> buckets = getBuckets(pairs, 100L, 2, 200L, Long.MAX_VALUE);
         assertEquals(2, buckets.size());
 
         for (List<String> bucket : buckets)
@@ -151,7 +162,7 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
                 Pair.create("b", 3899L),
                 Pair.create("c", 3900L)
         );
-        buckets = getBuckets(pairs, 100L, 3, 4050L);
+        buckets = getBuckets(pairs, 100L, 3, 4050L, Long.MAX_VALUE);
         // targets (divPosition, size): (40, 100), (39, 100), (12, 300), (3, 900), (0, 2700)
         // in other words: 0 - 2699, 2700 - 3599, 3600 - 3899, 3900 - 3999, 4000 - 4099
         assertEquals(3, buckets.size());
@@ -177,7 +188,7 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
                 Pair.create("e", 3950L),
                 Pair.create("too new", 4125L)
         );
-        buckets = getBuckets(pairs, 100L, 1, 4050L);
+        buckets = getBuckets(pairs, 100L, 1, 4050L, Long.MAX_VALUE);
 
         assertEquals(5, buckets.size());
 
@@ -193,7 +204,6 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
-        cfs.truncateBlocking();
         cfs.disableAutoCompaction();
 
         ByteBuffer value = ByteBuffer.wrap(new byte[100]);
@@ -212,15 +222,16 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
 
         List<SSTableReader> sstrs = new ArrayList<>(cfs.getSSTables());
 
-        List<SSTableReader> newBucket = newestBucket(Collections.singletonList(sstrs.subList(0, 2)), 4, 32, 9, 10, new SizeTieredCompactionStrategyOptions());
+        List<SSTableReader> newBucket = newestBucket(Collections.singletonList(sstrs.subList(0, 2)), 4, 32, 9, 10, Long.MAX_VALUE, new SizeTieredCompactionStrategyOptions());
         assertTrue("incoming bucket should not be accepted when it has below the min threshold SSTables", newBucket.isEmpty());
 
-        newBucket = newestBucket(Collections.singletonList(sstrs.subList(0, 2)), 4, 32, 10, 10, new SizeTieredCompactionStrategyOptions());
+        newBucket = newestBucket(Collections.singletonList(sstrs.subList(0, 2)), 4, 32, 10, 10, Long.MAX_VALUE, new SizeTieredCompactionStrategyOptions());
         assertFalse("non-incoming bucket should be accepted when it has at least 2 SSTables", newBucket.isEmpty());
 
         assertEquals("an sstable with a single value should have equal min/max timestamps", sstrs.get(0).getMinTimestamp(), sstrs.get(0).getMaxTimestamp());
         assertEquals("an sstable with a single value should have equal min/max timestamps", sstrs.get(1).getMinTimestamp(), sstrs.get(1).getMaxTimestamp());
         assertEquals("an sstable with a single value should have equal min/max timestamps", sstrs.get(2).getMinTimestamp(), sstrs.get(2).getMaxTimestamp());
+        cfs.truncateBlocking();
     }
 
     @Test
@@ -228,7 +239,6 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
-        cfs.truncateBlocking();
         cfs.disableAutoCompaction();
 
         ByteBuffer value = ByteBuffer.wrap(new byte[100]);
@@ -259,6 +269,7 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
 
         filtered = filterOldSSTables(sstrs, 1, 4);
         assertEquals("no sstables should remain when all are too old", 0, Iterables.size(filtered));
+        cfs.truncateBlocking();
     }
 
 
@@ -267,7 +278,6 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
-        cfs.truncateBlocking();
         cfs.disableAutoCompaction();
 
         ByteBuffer value = ByteBuffer.wrap(new byte[100]);
@@ -305,6 +315,7 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
         SSTableReader sstable = t.sstables.iterator().next();
         assertEquals(sstable, expiredSSTable);
         cfs.getDataTracker().unmarkCompacting(cfs.getSSTables());
+        cfs.truncateBlocking();
     }
 
     @Test
@@ -312,7 +323,6 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
-        cfs.truncateBlocking();
         cfs.disableAutoCompaction();
         ByteBuffer bigValue = ByteBuffer.wrap(new byte[10000]);
         ByteBuffer value = ByteBuffer.wrap(new byte[100]);
@@ -344,7 +354,9 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
         DateTieredCompactionStrategy dtcs = new DateTieredCompactionStrategy(cfs, options);
         for (SSTableReader sstable : cfs.getSSTables())
             dtcs.addSSTable(sstable);
-        assertEquals(20, dtcs.getNextBackgroundTask(0).sstables.size());
+        AbstractCompactionTask task = dtcs.getNextBackgroundTask(0);
+        assertEquals(20, task.sstables.size());
+        cfs.getDataTracker().unmarkCompacting(task.sstables);
+        cfs.truncateBlocking();
     }
-
 }


[2/5] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by ma...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2


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

Branch: refs/heads/trunk
Commit: 073f06262a60f661d3db5e308daa9877516cda51
Parents: 95feab6 99617a5
Author: Marcus Eriksson <ma...@apache.org>
Authored: Fri Nov 20 11:05:01 2015 +0100
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Fri Nov 20 11:05:01 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 pylib/cqlshlib/cql3handling.py                  |  1 +
 .../DateTieredCompactionStrategy.java           | 34 ++++++++++--------
 .../DateTieredCompactionStrategyOptions.java    | 30 ++++++++++++++--
 .../DateTieredCompactionStrategyTest.java       | 37 +++++++++++++-------
 5 files changed, 73 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/073f0626/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 14e8c3d,66423c7..867226f
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,17 -1,5 +1,18 @@@
 -2.1.12
 +2.2.4
 + * Don't do anticompaction after subrange repair (CASSANDRA-10422)
 + * Fix SimpleDateType type compatibility (CASSANDRA-10027)
 + * (Hadoop) fix splits calculation (CASSANDRA-10640)
 + * (Hadoop) ensure that Cluster instances are always closed (CASSANDRA-10058)
 + * (cqlsh) show partial trace if incomplete after max_trace_wait (CASSANDRA-7645)
 + * Use most up-to-date version of schema for system tables (CASSANDRA-10652)
 + * Deprecate memory_allocator in cassandra.yaml (CASSANDRA-10581,10628)
 + * Expose phi values from failure detector via JMX and tweak debug
 +   and trace logging (CASSANDRA-9526)
 + * Fix RangeNamesQueryPager (CASSANDRA-10509)
 + * Deprecate Pig support (CASSANDRA-10542)
 + * Reduce contention getting instances of CompositeType (CASSANDRA-10433)
 +Merged from 2.1:
+  * Limit window size in DTCS (CASSANDRA-10280)
   * sstableloader does not use MAX_HEAP_SIZE env parameter (CASSANDRA-10188)
   * (cqlsh) Improve COPY TO performance and error handling (CASSANDRA-9304)
   * Don't remove level info when running upgradesstables (CASSANDRA-10692)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/073f0626/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/073f0626/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/073f0626/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
index 2824d3a,5afd575..2fab014
--- a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
@@@ -312,10 -311,11 +322,11 @@@ public class DateTieredCompactionStrate
          Thread.sleep(2000);
          AbstractCompactionTask t = dtcs.getNextBackgroundTask((int) (System.currentTimeMillis()/1000));
          assertNotNull(t);
 -        assertEquals(1, Iterables.size(t.sstables));
 -        SSTableReader sstable = t.sstables.iterator().next();
 +        assertEquals(1, Iterables.size(t.transaction.originals()));
 +        SSTableReader sstable = t.transaction.originals().iterator().next();
          assertEquals(sstable, expiredSSTable);
 -        cfs.getDataTracker().unmarkCompacting(cfs.getSSTables());
 +        t.transaction.abort();
+         cfs.truncateBlocking();
      }
  
      @Test
@@@ -355,7 -354,9 +365,8 @@@
          DateTieredCompactionStrategy dtcs = new DateTieredCompactionStrategy(cfs, options);
          for (SSTableReader sstable : cfs.getSSTables())
              dtcs.addSSTable(sstable);
-         assertEquals(20, dtcs.getNextBackgroundTask(0).transaction.originals().size());
+         AbstractCompactionTask task = dtcs.getNextBackgroundTask(0);
 -        assertEquals(20, task.sstables.size());
 -        cfs.getDataTracker().unmarkCompacting(task.sstables);
 -        cfs.truncateBlocking();
++        assertEquals(20, task.transaction.originals().size());
++        task.transaction.abort();
      }
- 
  }


[3/5] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by ma...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/trunk
Commit: f4ae344871bbf6a57e4e4d40ec92527a172402a7
Parents: 747e5fd 073f062
Author: Marcus Eriksson <ma...@apache.org>
Authored: Fri Nov 20 11:06:48 2015 +0100
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Fri Nov 20 11:06:48 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 pylib/cqlshlib/cql3handling.py                  |  1 +
 .../DateTieredCompactionStrategy.java           | 34 +++++++++++---------
 .../DateTieredCompactionStrategyOptions.java    | 30 +++++++++++++++--
 .../DateTieredCompactionStrategyTest.java       | 34 ++++++++++++--------
 5 files changed, 70 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4ae3448/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 5e44f27,867226f..c3469bc
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -9,10 -3,21 +9,11 @@@ Merged from 2.2
   * Fix SimpleDateType type compatibility (CASSANDRA-10027)
   * (Hadoop) fix splits calculation (CASSANDRA-10640)
   * (Hadoop) ensure that Cluster instances are always closed (CASSANDRA-10058)
 - * (cqlsh) show partial trace if incomplete after max_trace_wait (CASSANDRA-7645)
 - * Use most up-to-date version of schema for system tables (CASSANDRA-10652)
 - * Deprecate memory_allocator in cassandra.yaml (CASSANDRA-10581,10628)
 - * Expose phi values from failure detector via JMX and tweak debug
 -   and trace logging (CASSANDRA-9526)
 - * Fix RangeNamesQueryPager (CASSANDRA-10509)
 - * Deprecate Pig support (CASSANDRA-10542)
 - * Reduce contention getting instances of CompositeType (CASSANDRA-10433)
  Merged from 2.1:
+  * Limit window size in DTCS (CASSANDRA-10280)
   * sstableloader does not use MAX_HEAP_SIZE env parameter (CASSANDRA-10188)
   * (cqlsh) Improve COPY TO performance and error handling (CASSANDRA-9304)
 - * Don't remove level info when running upgradesstables (CASSANDRA-10692)
   * Create compression chunk for sending file only (CASSANDRA-10680)
 - * Make buffered read size configurable (CASSANDRA-10249)
   * Forbid compact clustering column type changes in ALTER TABLE (CASSANDRA-8879)
   * Reject incremental repair with subrange repair (CASSANDRA-10422)
   * Add a nodetool command to refresh size_estimates (CASSANDRA-9579)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4ae3448/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4ae3448/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4ae3448/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
index 01a6dfa,2fab014..22b4829
--- a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
@@@ -221,12 -231,12 +231,12 @@@ public class DateTieredCompactionStrate
          }
          cfs.forceBlockingFlush();
  
 -        List<SSTableReader> sstrs = new ArrayList<>(cfs.getSSTables());
 +        List<SSTableReader> sstrs = new ArrayList<>(cfs.getLiveSSTables());
  
-         List<SSTableReader> newBucket = newestBucket(Collections.singletonList(sstrs.subList(0, 2)), 4, 32, 9, 10, new SizeTieredCompactionStrategyOptions());
+         List<SSTableReader> newBucket = newestBucket(Collections.singletonList(sstrs.subList(0, 2)), 4, 32, 9, 10, Long.MAX_VALUE, new SizeTieredCompactionStrategyOptions());
          assertTrue("incoming bucket should not be accepted when it has below the min threshold SSTables", newBucket.isEmpty());
  
-         newBucket = newestBucket(Collections.singletonList(sstrs.subList(0, 2)), 4, 32, 10, 10, new SizeTieredCompactionStrategyOptions());
+         newBucket = newestBucket(Collections.singletonList(sstrs.subList(0, 2)), 4, 32, 10, 10, Long.MAX_VALUE, new SizeTieredCompactionStrategyOptions());
          assertFalse("non-incoming bucket should be accepted when it has at least 2 SSTables", newBucket.isEmpty());
  
          assertEquals("an sstable with a single value should have equal min/max timestamps", sstrs.get(0).getMinTimestamp(), sstrs.get(0).getMaxTimestamp());


[5/5] cassandra git commit: Merge branch 'cassandra-3.1' into trunk

Posted by ma...@apache.org.
Merge branch 'cassandra-3.1' into trunk


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

Branch: refs/heads/trunk
Commit: 0caa17de0eb0cb96b463093bd5fa4e1cbefd0b07
Parents: f81a91d 36019ab
Author: Marcus Eriksson <ma...@apache.org>
Authored: Fri Nov 20 11:07:09 2015 +0100
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Fri Nov 20 11:07:09 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 pylib/cqlshlib/cql3handling.py                  |  1 +
 .../DateTieredCompactionStrategy.java           | 34 +++++++++++---------
 .../DateTieredCompactionStrategyOptions.java    | 30 +++++++++++++++--
 .../DateTieredCompactionStrategyTest.java       | 34 ++++++++++++--------
 5 files changed, 70 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0caa17de/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0caa17de/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------


[4/5] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.1

Posted by ma...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.1


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

Branch: refs/heads/trunk
Commit: 36019ab1728eb084b201ec452d17047555ebc1ef
Parents: 28e51d6 f4ae344
Author: Marcus Eriksson <ma...@apache.org>
Authored: Fri Nov 20 11:06:58 2015 +0100
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Fri Nov 20 11:06:58 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 pylib/cqlshlib/cql3handling.py                  |  1 +
 .../DateTieredCompactionStrategy.java           | 34 +++++++++++---------
 .../DateTieredCompactionStrategyOptions.java    | 30 +++++++++++++++--
 .../DateTieredCompactionStrategyTest.java       | 34 ++++++++++++--------
 5 files changed, 70 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/36019ab1/CHANGES.txt
----------------------------------------------------------------------