You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by st...@apache.org on 2017/01/11 07:01:53 UTC

[01/30] cassandra git commit: Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 bdca25e96 -> 47f7d9c6a
  refs/heads/cassandra-2.2 dbe039f62 -> 2acc15ba3
  refs/heads/cassandra-3.0 d8eec1cd6 -> 6ff56c30e
  refs/heads/cassandra-3.11 e9251e1fb -> b52c82045
  refs/heads/cassandra-3.X d1d75ad8c -> 699eff071
  refs/heads/trunk b3ffdf8c4 -> 87b0b36c7


Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

patch by Jon Haddad; reviewed by Nate McCall for CASSANDRA-12979


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

Branch: refs/heads/cassandra-3.0
Commit: dbe039f628220e5141836ecb9aab05d4da60589a
Parents: 6f360b6
Author: Jon Haddad <jo...@jonhaddad.com>
Authored: Wed Jan 4 11:52:54 2017 -0800
Committer: Nate McCall <zz...@gmail.com>
Committed: Tue Jan 10 15:15:47 2017 +1300

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  6 +++
 .../cassandra/db/compaction/CompactionTask.java | 40 +++++++++++++++-----
 3 files changed, 38 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbe039f6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b41313d..4b301ee 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -24,6 +24,7 @@
  * Make Collections deserialization more robust (CASSANDRA-12618)
  * Better handle invalid system roles table (CASSANDRA-12700)
  * Split consistent range movement flag correction (CASSANDRA-12786)
+ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 Merged from 2.1:
  * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
  * Don't skip sstables based on maxLocalDeletionTime (CASSANDRA-12765)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbe039f6/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 37949a1..06e9eeb 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -16,6 +16,12 @@ using the provided 'sstableupgrade' tool.
 2.2.9
 =====
 
+Upgrading
+---------
+   - Compaction now correctly drops sstables out of CompactionTask when there
+     isn't enough disk space to perform the full compaction.  This should reduce
+     pending compaction tasks on systems with little remaining disk space.
+
 Deprecation
 -----------
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbe039f6/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 20d3dc0..7489b3d 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -75,12 +75,14 @@ public class CompactionTask extends AbstractCompactionTask
         return transaction.originals().size();
     }
 
-    public boolean reduceScopeForLimitedSpace()
+    public boolean reduceScopeForLimitedSpace(long expectedSize)
     {
         if (partialCompactionsAcceptable() && transaction.originals().size() > 1)
         {
             // Try again w/o the largest one.
-            logger.warn("insufficient space to compact all requested files {}", StringUtils.join(transaction.originals(), ", "));
+            logger.warn("insufficient space to compact all requested files. {}MB required, {}",
+                        (float) expectedSize / 1024 / 1024,
+                        StringUtils.join(transaction.originals(), ", "));
             // Note that we have removed files that are still marked as compacting.
             // This suboptimal but ok since the caller will unmark all the sstables at the end.
             SSTableReader removedSSTable = cfs.getMaxSizeFile(transaction.originals());
@@ -113,9 +115,8 @@ public class CompactionTask extends AbstractCompactionTask
 
         // note that we need to do a rough estimate early if we can fit the compaction on disk - this is pessimistic, but
         // since we might remove sstables from the compaction in checkAvailableDiskSpace it needs to be done here
-        long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
-        long earlySSTableEstimate = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
-        checkAvailableDiskSpace(earlySSTableEstimate, expectedWriteSize);
+
+        checkAvailableDiskSpace();
 
         // sanity check: all sstables must belong to the same cfs
         assert !Iterables.any(transaction.originals(), new Predicate<SSTableReader>()
@@ -271,12 +272,33 @@ public class CompactionTask extends AbstractCompactionTask
         return minRepairedAt;
     }
 
-    protected void checkAvailableDiskSpace(long estimatedSSTables, long expectedWriteSize)
+    /*
+    Checks if we have enough disk space to execute the compaction.  Drops the largest sstable out of the Task until
+    there's enough space (in theory) to handle the compaction.  Does not take into account space that will be taken by
+    other compactions.
+     */
+    protected void checkAvailableDiskSpace()
     {
-        while (!cfs.directories.hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+        AbstractCompactionStrategy strategy = cfs.getCompactionStrategy();
+
+        while(true)
         {
-            if (!reduceScopeForLimitedSpace())
-                throw new RuntimeException(String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize));
+            long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
+            long estimatedSSTables = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
+
+            if(cfs.directories.hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+                break;
+
+            if (!reduceScopeForLimitedSpace(expectedWriteSize))
+            {
+                // we end up here if we can't take any more sstables out of the compaction.
+                // usually means we've run out of disk space
+                String msg = String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize);
+                logger.warn(msg);
+                throw new RuntimeException(msg);
+            }
+            logger.warn("Not enough space for compaction, {}MB estimated.  Reducing scope.",
+                            (float) expectedWriteSize / 1024 / 1024);
         }
     }
 


[27/30] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by st...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


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

Branch: refs/heads/trunk
Commit: b52c820454619e2c007c8d312482e860132aae5c
Parents: e9251e1 6ff56c3
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jan 11 07:52:03 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:52:03 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 3 ++-
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b52c8204/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 99f6613,5f37cd7..b8250ab
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,117 -1,6 +1,116 @@@
 -3.0.11
 +3.10
 + * Fixed query monitoring for range queries (CASSANDRA-13050)
 + * Remove outboundBindAny configuration property (CASSANDRA-12673)
 + * Use correct bounds for all-data range when filtering (CASSANDRA-12666)
 + * Remove timing window in test case (CASSANDRA-12875)
 + * Resolve unit testing without JCE security libraries installed (CASSANDRA-12945)
 + * Fix inconsistencies in cassandra-stress load balancing policy (CASSANDRA-12919)
 + * Fix validation of non-frozen UDT cells (CASSANDRA-12916)
 + * Don't shut down socket input/output on StreamSession (CASSANDRA-12903)
 + * Fix Murmur3PartitionerTest (CASSANDRA-12858)
 + * Move cqlsh syntax rules into separate module and allow easier customization (CASSANDRA-12897)
 + * Fix CommitLogSegmentManagerTest (CASSANDRA-12283)
 + * Fix cassandra-stress truncate option (CASSANDRA-12695)
 + * Fix crossNode value when receiving messages (CASSANDRA-12791)
 + * Don't load MX4J beans twice (CASSANDRA-12869)
 + * Extend native protocol request flags, add versions to SUPPORTED, and introduce ProtocolVersion enum (CASSANDRA-12838)
 + * Set JOINING mode when running pre-join tasks (CASSANDRA-12836)
 + * remove net.mintern.primitive library due to license issue (CASSANDRA-12845)
 + * Properly format IPv6 addresses when logging JMX service URL (CASSANDRA-12454)
 + * Optimize the vnode allocation for single replica per DC (CASSANDRA-12777)
 + * Use non-token restrictions for bounds when token restrictions are overridden (CASSANDRA-12419)
 + * Fix CQLSH auto completion for PER PARTITION LIMIT (CASSANDRA-12803)
 + * Use different build directories for Eclipse and Ant (CASSANDRA-12466)
 + * Avoid potential AttributeError in cqlsh due to no table metadata (CASSANDRA-12815)
 + * Fix RandomReplicationAwareTokenAllocatorTest.testExistingCluster (CASSANDRA-12812)
 + * Upgrade commons-codec to 1.9 (CASSANDRA-12790)
 + * Make the fanout size for LeveledCompactionStrategy to be configurable (CASSANDRA-11550)
 + * Add duration data type (CASSANDRA-11873)
 + * Fix timeout in ReplicationAwareTokenAllocatorTest (CASSANDRA-12784)
 + * Improve sum aggregate functions (CASSANDRA-12417)
 + * Make cassandra.yaml docs for batch_size_*_threshold_in_kb reflect changes in CASSANDRA-10876 (CASSANDRA-12761)
 + * cqlsh fails to format collections when using aliases (CASSANDRA-11534)
 + * Check for hash conflicts in prepared statements (CASSANDRA-12733)
 + * Exit query parsing upon first error (CASSANDRA-12598)
 + * Fix cassandra-stress to use single seed in UUID generation (CASSANDRA-12729)
 + * CQLSSTableWriter does not allow Update statement (CASSANDRA-12450)
 + * Config class uses boxed types but DD exposes primitive types (CASSANDRA-12199)
 + * Add pre- and post-shutdown hooks to Storage Service (CASSANDRA-12461)
 + * Add hint delivery metrics (CASSANDRA-12693)
 + * Remove IndexInfo cache from FileIndexInfoRetriever (CASSANDRA-12731)
 + * ColumnIndex does not reuse buffer (CASSANDRA-12502)
 + * cdc column addition still breaks schema migration tasks (CASSANDRA-12697)
 + * Upgrade metrics-reporter dependencies (CASSANDRA-12089)
 + * Tune compaction thread count via nodetool (CASSANDRA-12248)
 + * Add +=/-= shortcut syntax for update queries (CASSANDRA-12232)
 + * Include repair session IDs in repair start message (CASSANDRA-12532)
 + * Add a blocking task to Index, run before joining the ring (CASSANDRA-12039)
 + * Fix NPE when using CQLSSTableWriter (CASSANDRA-12667)
 + * Support optional backpressure strategies at the coordinator (CASSANDRA-9318)
 + * Make randompartitioner work with new vnode allocation (CASSANDRA-12647)
 + * Fix cassandra-stress graphing (CASSANDRA-12237)
 + * Allow filtering on partition key columns for queries without secondary indexes (CASSANDRA-11031)
 + * Fix Cassandra Stress reporting thread model and precision (CASSANDRA-12585)
 + * Add JMH benchmarks.jar (CASSANDRA-12586)
 + * Cleanup uses of AlterTableStatementColumn (CASSANDRA-12567)
 + * Add keep-alive to streaming (CASSANDRA-11841)
 + * Tracing payload is passed through newSession(..) (CASSANDRA-11706)
 + * avoid deleting non existing sstable files and improve related log messages (CASSANDRA-12261)
 + * json/yaml output format for nodetool compactionhistory (CASSANDRA-12486)
 + * Retry all internode messages once after a connection is
 +   closed and reopened (CASSANDRA-12192)
 + * Add support to rebuild from targeted replica (CASSANDRA-9875)
 + * Add sequence distribution type to cassandra stress (CASSANDRA-12490)
 + * "SELECT * FROM foo LIMIT ;" does not error out (CASSANDRA-12154)
 + * Define executeLocally() at the ReadQuery Level (CASSANDRA-12474)
 + * Extend read/write failure messages with a map of replica addresses
 +   to error codes in the v5 native protocol (CASSANDRA-12311)
 + * Fix rebuild of SASI indexes with existing index files (CASSANDRA-12374)
 + * Let DatabaseDescriptor not implicitly startup services (CASSANDRA-9054, 12550)
 + * Fix clustering indexes in presence of static columns in SASI (CASSANDRA-12378)
 + * Fix queries on columns with reversed type on SASI indexes (CASSANDRA-12223)
 + * Added slow query log (CASSANDRA-12403)
 + * Count full coordinated request against timeout (CASSANDRA-12256)
 + * Allow TTL with null value on insert and update (CASSANDRA-12216)
 + * Make decommission operation resumable (CASSANDRA-12008)
 + * Add support to one-way targeted repair (CASSANDRA-9876)
 + * Remove clientutil jar (CASSANDRA-11635)
 + * Fix compaction throughput throttle (CASSANDRA-12366, CASSANDRA-12717)
 + * Delay releasing Memtable memory on flush until PostFlush has finished running (CASSANDRA-12358)
 + * Cassandra stress should dump all setting on startup (CASSANDRA-11914)
 + * Make it possible to compact a given token range (CASSANDRA-10643)
 + * Allow updating DynamicEndpointSnitch properties via JMX (CASSANDRA-12179)
 + * Collect metrics on queries by consistency level (CASSANDRA-7384)
 + * Add support for GROUP BY to SELECT statement (CASSANDRA-10707)
 + * Deprecate memtable_cleanup_threshold and update default for memtable_flush_writers (CASSANDRA-12228)
 + * Upgrade to OHC 0.4.4 (CASSANDRA-12133)
 + * Add version command to cassandra-stress (CASSANDRA-12258)
 + * Create compaction-stress tool (CASSANDRA-11844)
 + * Garbage-collecting compaction operation and schema option (CASSANDRA-7019)
 + * Add beta protocol flag for v5 native protocol (CASSANDRA-12142)
 + * Support filtering on non-PRIMARY KEY columns in the CREATE
 +   MATERIALIZED VIEW statement's WHERE clause (CASSANDRA-10368)
 + * Unify STDOUT and SYSTEMLOG logback format (CASSANDRA-12004)
 + * COPY FROM should raise error for non-existing input files (CASSANDRA-12174)
 + * Faster write path (CASSANDRA-12269)
 + * Option to leave omitted columns in INSERT JSON unset (CASSANDRA-11424)
 + * Support json/yaml output in nodetool tpstats (CASSANDRA-12035)
 + * Expose metrics for successful/failed authentication attempts (CASSANDRA-10635)
 + * Prepend snapshot name with "truncated" or "dropped" when a snapshot
 +   is taken before truncating or dropping a table (CASSANDRA-12178)
 + * Optimize RestrictionSet (CASSANDRA-12153)
 + * cqlsh does not automatically downgrade CQL version (CASSANDRA-12150)
 + * Omit (de)serialization of state variable in UDAs (CASSANDRA-9613)
 + * Create a system table to expose prepared statements (CASSANDRA-8831)
 + * Reuse DataOutputBuffer from ColumnIndex (CASSANDRA-11970)
 + * Remove DatabaseDescriptor dependency from SegmentedFile (CASSANDRA-11580)
 + * Add supplied username to authentication error messages (CASSANDRA-12076)
 + * Remove pre-startup check for open JMX port (CASSANDRA-12074)
 + * Remove compaction Severity from DynamicEndpointSnitch (CASSANDRA-11738)
 + * Restore resumable hints delivery (CASSANDRA-11960)
 + * Properly report LWT contention (CASSANDRA-12626)
 +Merged from 3.0:
-  * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
   * Replace empty strings with null values if they cannot be converted (CASSANDRA-12794)
 - * Fixed flacky SSTableRewriterTest: check file counts before calling validateCFS (CASSANDRA-12348)
   * Fix deserialization of 2.x DeletedCells (CASSANDRA-12620)
   * Add parent repair session id to anticompaction log message (CASSANDRA-12186)
   * Improve contention handling on failure to acquire MV lock for streaming and hints (CASSANDRA-12905)
@@@ -177,16 -73,6 +176,18 @@@
   * Correct log message for statistics of offheap memtable flush (CASSANDRA-12776)
   * Explicitly set locale for string validation (CASSANDRA-12541,CASSANDRA-12542,CASSANDRA-12543,CASSANDRA-12545)
  Merged from 2.2:
++ * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
++ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 + * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
 + * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
 + * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)
 + * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
 + * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
 + * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
 + * Use saved tokens when setting local tokens on StorageService.joinRing (CASSANDRA-12935)
 + * cqlsh: fix DESC TYPES errors (CASSANDRA-12914)
 + * Fix leak on skipped SSTables in sstableupgrade (CASSANDRA-12899)
 + * Avoid blocking gossip during pending range calculation (CASSANDRA-12281)
   * Fix purgeability of tombstones with max timestamp (CASSANDRA-12792)
   * Fail repair if participant dies during sync or anticompaction (CASSANDRA-12901)
   * cqlsh COPY: unprotected pk values before converting them if not using prepared statements (CASSANDRA-12863)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b52c8204/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index d1cdaed,efa9330..31aa689
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@@ -75,14 -75,15 +75,16 @@@ public class CustomTThreadPoolServer ex
      private final TThreadPoolServer.Args args;
  
      //Track and Limit the number of connected clients
-     private final AtomicInteger activeClients = new AtomicInteger(0);
+     private final AtomicInteger activeClients;
  
  
 -    public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService) {
 +    public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService)
 +    {
          super(args);
          this.executorService = executorService;
+         this.stopped = false;
          this.args = args;
+         this.activeClients = new AtomicInteger(0);
      }
  
      @SuppressWarnings("resource")


[06/30] cassandra git commit: Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

Posted by st...@apache.org.
Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

patch by Jon Haddad; reviewed by Nate McCall for CASSANDRA-12979


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

Branch: refs/heads/cassandra-3.11
Commit: d8eec1cd6d095961b3754b0de41ffc9061cba370
Parents: 65bd455
Author: Jon Haddad <jo...@jonhaddad.com>
Authored: Wed Jan 4 11:52:54 2017 -0800
Committer: Nate McCall <zz...@gmail.com>
Committed: Tue Jan 10 15:24:36 2017 +1300

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 NEWS.txt                                        |  3 ++
 .../cassandra/db/compaction/CompactionTask.java | 40 +++++++++++++++-----
 3 files changed, 36 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d8eec1cd/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 11b0482..a533e6f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -18,6 +18,7 @@
  * AnticompactionRequestSerializer serializedSize is incorrect (CASSANDRA-12934)
  * Prevent reloading of logback.xml from UDF sandbox (CASSANDRA-12535)
  * Reenable HeapPool (CASSANDRA-12900)
+ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 Merged from 2.2:
  * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
  * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
@@ -34,6 +35,7 @@ Merged from 2.1:
  * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
 
 
+
 3.0.10
  * Disallow offheap_buffers memtable allocation (CASSANDRA-11039)
  * Fix CommitLogSegmentManagerTest (CASSANDRA-12283)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d8eec1cd/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index b4e6551..3b8a333 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -26,6 +26,9 @@ Upgrading
    - Only Java and JavaScript are now supported UDF languages.
      The sandbox in 3.0 already prevented the use of script languages except Java
      and JavaScript.
+   - Compaction now correctly drops sstables out of CompactionTask when there
+     isn't enough disk space to perform the full compaction.  This should reduce
+     pending compaction tasks on systems with little remaining disk space.
 
 3.0.10
 =====

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d8eec1cd/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index f0a1f47..1636ab9 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -90,12 +90,14 @@ public class CompactionTask extends AbstractCompactionTask
         return transaction.originals().size();
     }
 
-    public boolean reduceScopeForLimitedSpace()
+    public boolean reduceScopeForLimitedSpace(long expectedSize)
     {
         if (partialCompactionsAcceptable() && transaction.originals().size() > 1)
         {
             // Try again w/o the largest one.
-            logger.warn("insufficient space to compact all requested files {}", StringUtils.join(transaction.originals(), ", "));
+            logger.warn("insufficient space to compact all requested files. {}MB required, {}",
+                        (float) expectedSize / 1024 / 1024,
+                        StringUtils.join(transaction.originals(), ", "));
             // Note that we have removed files that are still marked as compacting.
             // This suboptimal but ok since the caller will unmark all the sstables at the end.
             SSTableReader removedSSTable = cfs.getMaxSizeFile(transaction.originals());
@@ -128,9 +130,8 @@ public class CompactionTask extends AbstractCompactionTask
 
         // note that we need to do a rough estimate early if we can fit the compaction on disk - this is pessimistic, but
         // since we might remove sstables from the compaction in checkAvailableDiskSpace it needs to be done here
-        long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
-        long earlySSTableEstimate = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
-        checkAvailableDiskSpace(earlySSTableEstimate, expectedWriteSize);
+
+        checkAvailableDiskSpace();
 
         // sanity check: all sstables must belong to the same cfs
         assert !Iterables.any(transaction.originals(), new Predicate<SSTableReader>()
@@ -283,7 +284,12 @@ public class CompactionTask extends AbstractCompactionTask
         return minRepairedAt;
     }
 
-    protected void checkAvailableDiskSpace(long estimatedSSTables, long expectedWriteSize)
+    /*
+    Checks if we have enough disk space to execute the compaction.  Drops the largest sstable out of the Task until
+    there's enough space (in theory) to handle the compaction.  Does not take into account space that will be taken by
+    other compactions.
+     */
+    protected void checkAvailableDiskSpace()
     {
         if(!cfs.isCompactionDiskSpaceCheckEnabled() && compactionType == OperationType.COMPACTION)
         {
@@ -291,10 +297,26 @@ public class CompactionTask extends AbstractCompactionTask
             return;
         }
 
-        while (!getDirectories().hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+        CompactionStrategyManager strategy = cfs.getCompactionStrategyManager();
+
+        while(true)
         {
-            if (!reduceScopeForLimitedSpace())
-                throw new RuntimeException(String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize));
+            long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
+            long estimatedSSTables = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
+
+            if(cfs.getDirectories().hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+                break;
+
+            if (!reduceScopeForLimitedSpace(expectedWriteSize))
+            {
+                // we end up here if we can't take any more sstables out of the compaction.
+                // usually means we've run out of disk space
+                String msg = String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize);
+                logger.warn(msg);
+                throw new RuntimeException(msg);
+            }
+            logger.warn("Not enough space for compaction, {}MB estimated.  Reducing scope.",
+                            (float) expectedWriteSize / 1024 / 1024);
         }
     }
 


[25/30] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by st...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


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

Branch: refs/heads/cassandra-3.X
Commit: b52c820454619e2c007c8d312482e860132aae5c
Parents: e9251e1 6ff56c3
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jan 11 07:52:03 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:52:03 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 3 ++-
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b52c8204/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 99f6613,5f37cd7..b8250ab
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,117 -1,6 +1,116 @@@
 -3.0.11
 +3.10
 + * Fixed query monitoring for range queries (CASSANDRA-13050)
 + * Remove outboundBindAny configuration property (CASSANDRA-12673)
 + * Use correct bounds for all-data range when filtering (CASSANDRA-12666)
 + * Remove timing window in test case (CASSANDRA-12875)
 + * Resolve unit testing without JCE security libraries installed (CASSANDRA-12945)
 + * Fix inconsistencies in cassandra-stress load balancing policy (CASSANDRA-12919)
 + * Fix validation of non-frozen UDT cells (CASSANDRA-12916)
 + * Don't shut down socket input/output on StreamSession (CASSANDRA-12903)
 + * Fix Murmur3PartitionerTest (CASSANDRA-12858)
 + * Move cqlsh syntax rules into separate module and allow easier customization (CASSANDRA-12897)
 + * Fix CommitLogSegmentManagerTest (CASSANDRA-12283)
 + * Fix cassandra-stress truncate option (CASSANDRA-12695)
 + * Fix crossNode value when receiving messages (CASSANDRA-12791)
 + * Don't load MX4J beans twice (CASSANDRA-12869)
 + * Extend native protocol request flags, add versions to SUPPORTED, and introduce ProtocolVersion enum (CASSANDRA-12838)
 + * Set JOINING mode when running pre-join tasks (CASSANDRA-12836)
 + * remove net.mintern.primitive library due to license issue (CASSANDRA-12845)
 + * Properly format IPv6 addresses when logging JMX service URL (CASSANDRA-12454)
 + * Optimize the vnode allocation for single replica per DC (CASSANDRA-12777)
 + * Use non-token restrictions for bounds when token restrictions are overridden (CASSANDRA-12419)
 + * Fix CQLSH auto completion for PER PARTITION LIMIT (CASSANDRA-12803)
 + * Use different build directories for Eclipse and Ant (CASSANDRA-12466)
 + * Avoid potential AttributeError in cqlsh due to no table metadata (CASSANDRA-12815)
 + * Fix RandomReplicationAwareTokenAllocatorTest.testExistingCluster (CASSANDRA-12812)
 + * Upgrade commons-codec to 1.9 (CASSANDRA-12790)
 + * Make the fanout size for LeveledCompactionStrategy to be configurable (CASSANDRA-11550)
 + * Add duration data type (CASSANDRA-11873)
 + * Fix timeout in ReplicationAwareTokenAllocatorTest (CASSANDRA-12784)
 + * Improve sum aggregate functions (CASSANDRA-12417)
 + * Make cassandra.yaml docs for batch_size_*_threshold_in_kb reflect changes in CASSANDRA-10876 (CASSANDRA-12761)
 + * cqlsh fails to format collections when using aliases (CASSANDRA-11534)
 + * Check for hash conflicts in prepared statements (CASSANDRA-12733)
 + * Exit query parsing upon first error (CASSANDRA-12598)
 + * Fix cassandra-stress to use single seed in UUID generation (CASSANDRA-12729)
 + * CQLSSTableWriter does not allow Update statement (CASSANDRA-12450)
 + * Config class uses boxed types but DD exposes primitive types (CASSANDRA-12199)
 + * Add pre- and post-shutdown hooks to Storage Service (CASSANDRA-12461)
 + * Add hint delivery metrics (CASSANDRA-12693)
 + * Remove IndexInfo cache from FileIndexInfoRetriever (CASSANDRA-12731)
 + * ColumnIndex does not reuse buffer (CASSANDRA-12502)
 + * cdc column addition still breaks schema migration tasks (CASSANDRA-12697)
 + * Upgrade metrics-reporter dependencies (CASSANDRA-12089)
 + * Tune compaction thread count via nodetool (CASSANDRA-12248)
 + * Add +=/-= shortcut syntax for update queries (CASSANDRA-12232)
 + * Include repair session IDs in repair start message (CASSANDRA-12532)
 + * Add a blocking task to Index, run before joining the ring (CASSANDRA-12039)
 + * Fix NPE when using CQLSSTableWriter (CASSANDRA-12667)
 + * Support optional backpressure strategies at the coordinator (CASSANDRA-9318)
 + * Make randompartitioner work with new vnode allocation (CASSANDRA-12647)
 + * Fix cassandra-stress graphing (CASSANDRA-12237)
 + * Allow filtering on partition key columns for queries without secondary indexes (CASSANDRA-11031)
 + * Fix Cassandra Stress reporting thread model and precision (CASSANDRA-12585)
 + * Add JMH benchmarks.jar (CASSANDRA-12586)
 + * Cleanup uses of AlterTableStatementColumn (CASSANDRA-12567)
 + * Add keep-alive to streaming (CASSANDRA-11841)
 + * Tracing payload is passed through newSession(..) (CASSANDRA-11706)
 + * avoid deleting non existing sstable files and improve related log messages (CASSANDRA-12261)
 + * json/yaml output format for nodetool compactionhistory (CASSANDRA-12486)
 + * Retry all internode messages once after a connection is
 +   closed and reopened (CASSANDRA-12192)
 + * Add support to rebuild from targeted replica (CASSANDRA-9875)
 + * Add sequence distribution type to cassandra stress (CASSANDRA-12490)
 + * "SELECT * FROM foo LIMIT ;" does not error out (CASSANDRA-12154)
 + * Define executeLocally() at the ReadQuery Level (CASSANDRA-12474)
 + * Extend read/write failure messages with a map of replica addresses
 +   to error codes in the v5 native protocol (CASSANDRA-12311)
 + * Fix rebuild of SASI indexes with existing index files (CASSANDRA-12374)
 + * Let DatabaseDescriptor not implicitly startup services (CASSANDRA-9054, 12550)
 + * Fix clustering indexes in presence of static columns in SASI (CASSANDRA-12378)
 + * Fix queries on columns with reversed type on SASI indexes (CASSANDRA-12223)
 + * Added slow query log (CASSANDRA-12403)
 + * Count full coordinated request against timeout (CASSANDRA-12256)
 + * Allow TTL with null value on insert and update (CASSANDRA-12216)
 + * Make decommission operation resumable (CASSANDRA-12008)
 + * Add support to one-way targeted repair (CASSANDRA-9876)
 + * Remove clientutil jar (CASSANDRA-11635)
 + * Fix compaction throughput throttle (CASSANDRA-12366, CASSANDRA-12717)
 + * Delay releasing Memtable memory on flush until PostFlush has finished running (CASSANDRA-12358)
 + * Cassandra stress should dump all setting on startup (CASSANDRA-11914)
 + * Make it possible to compact a given token range (CASSANDRA-10643)
 + * Allow updating DynamicEndpointSnitch properties via JMX (CASSANDRA-12179)
 + * Collect metrics on queries by consistency level (CASSANDRA-7384)
 + * Add support for GROUP BY to SELECT statement (CASSANDRA-10707)
 + * Deprecate memtable_cleanup_threshold and update default for memtable_flush_writers (CASSANDRA-12228)
 + * Upgrade to OHC 0.4.4 (CASSANDRA-12133)
 + * Add version command to cassandra-stress (CASSANDRA-12258)
 + * Create compaction-stress tool (CASSANDRA-11844)
 + * Garbage-collecting compaction operation and schema option (CASSANDRA-7019)
 + * Add beta protocol flag for v5 native protocol (CASSANDRA-12142)
 + * Support filtering on non-PRIMARY KEY columns in the CREATE
 +   MATERIALIZED VIEW statement's WHERE clause (CASSANDRA-10368)
 + * Unify STDOUT and SYSTEMLOG logback format (CASSANDRA-12004)
 + * COPY FROM should raise error for non-existing input files (CASSANDRA-12174)
 + * Faster write path (CASSANDRA-12269)
 + * Option to leave omitted columns in INSERT JSON unset (CASSANDRA-11424)
 + * Support json/yaml output in nodetool tpstats (CASSANDRA-12035)
 + * Expose metrics for successful/failed authentication attempts (CASSANDRA-10635)
 + * Prepend snapshot name with "truncated" or "dropped" when a snapshot
 +   is taken before truncating or dropping a table (CASSANDRA-12178)
 + * Optimize RestrictionSet (CASSANDRA-12153)
 + * cqlsh does not automatically downgrade CQL version (CASSANDRA-12150)
 + * Omit (de)serialization of state variable in UDAs (CASSANDRA-9613)
 + * Create a system table to expose prepared statements (CASSANDRA-8831)
 + * Reuse DataOutputBuffer from ColumnIndex (CASSANDRA-11970)
 + * Remove DatabaseDescriptor dependency from SegmentedFile (CASSANDRA-11580)
 + * Add supplied username to authentication error messages (CASSANDRA-12076)
 + * Remove pre-startup check for open JMX port (CASSANDRA-12074)
 + * Remove compaction Severity from DynamicEndpointSnitch (CASSANDRA-11738)
 + * Restore resumable hints delivery (CASSANDRA-11960)
 + * Properly report LWT contention (CASSANDRA-12626)
 +Merged from 3.0:
-  * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
   * Replace empty strings with null values if they cannot be converted (CASSANDRA-12794)
 - * Fixed flacky SSTableRewriterTest: check file counts before calling validateCFS (CASSANDRA-12348)
   * Fix deserialization of 2.x DeletedCells (CASSANDRA-12620)
   * Add parent repair session id to anticompaction log message (CASSANDRA-12186)
   * Improve contention handling on failure to acquire MV lock for streaming and hints (CASSANDRA-12905)
@@@ -177,16 -73,6 +176,18 @@@
   * Correct log message for statistics of offheap memtable flush (CASSANDRA-12776)
   * Explicitly set locale for string validation (CASSANDRA-12541,CASSANDRA-12542,CASSANDRA-12543,CASSANDRA-12545)
  Merged from 2.2:
++ * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
++ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 + * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
 + * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
 + * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)
 + * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
 + * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
 + * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
 + * Use saved tokens when setting local tokens on StorageService.joinRing (CASSANDRA-12935)
 + * cqlsh: fix DESC TYPES errors (CASSANDRA-12914)
 + * Fix leak on skipped SSTables in sstableupgrade (CASSANDRA-12899)
 + * Avoid blocking gossip during pending range calculation (CASSANDRA-12281)
   * Fix purgeability of tombstones with max timestamp (CASSANDRA-12792)
   * Fail repair if participant dies during sync or anticompaction (CASSANDRA-12901)
   * cqlsh COPY: unprotected pk values before converting them if not using prepared statements (CASSANDRA-12863)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b52c8204/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index d1cdaed,efa9330..31aa689
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@@ -75,14 -75,15 +75,16 @@@ public class CustomTThreadPoolServer ex
      private final TThreadPoolServer.Args args;
  
      //Track and Limit the number of connected clients
-     private final AtomicInteger activeClients = new AtomicInteger(0);
+     private final AtomicInteger activeClients;
  
  
 -    public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService) {
 +    public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService)
 +    {
          super(args);
          this.executorService = executorService;
+         this.stopped = false;
          this.args = args;
+         this.activeClients = new AtomicInteger(0);
      }
  
      @SuppressWarnings("resource")


[20/30] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by st...@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/2acc15ba
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2acc15ba
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2acc15ba

Branch: refs/heads/cassandra-3.X
Commit: 2acc15ba38695495acdb678dcb005d7f611bae2b
Parents: dbe039f 47f7d9c
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jan 11 07:37:22 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:37:22 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 1 +
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2acc15ba/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 4b301ee,23fd12a..a7e89ed
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,32 -1,8 +1,33 @@@
 -2.1.17
 +2.2.9
 + * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
 + * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
 + * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)
 + * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
 + * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
 + * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
 + * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)
 + * Use saved tokens when setting local tokens on StorageService.joinRing (CASSANDRA-12935)
 + * cqlsh: fix DESC TYPES errors (CASSANDRA-12914)
 + * Fix leak on skipped SSTables in sstableupgrade (CASSANDRA-12899)
 + * Avoid blocking gossip during pending range calculation (CASSANDRA-12281)
 + * Fix purgeability of tombstones with max timestamp (CASSANDRA-12792)
 + * Fail repair if participant dies during sync or anticompaction (CASSANDRA-12901)
 + * cqlsh COPY: unprotected pk values before converting them if not using prepared statements (CASSANDRA-12863)
 + * Fix Util.spinAssertEquals (CASSANDRA-12283)
 + * Fix potential NPE for compactionstats (CASSANDRA-12462)
 + * Prepare legacy authenticate statement if credentials table initialised after node startup (CASSANDRA-12813)
 + * Change cassandra.wait_for_tracing_events_timeout_secs default to 0 (CASSANDRA-12754)
 + * Clean up permissions when a UDA is dropped (CASSANDRA-12720)
 + * Limit colUpdateTimeDelta histogram updates to reasonable deltas (CASSANDRA-11117)
 + * Fix leak errors and execution rejected exceptions when draining (CASSANDRA-12457)
 + * Fix merkle tree depth calculation (CASSANDRA-12580)
 + * Make Collections deserialization more robust (CASSANDRA-12618)
 + * Better handle invalid system roles table (CASSANDRA-12700)
 + * Split consistent range movement flag correction (CASSANDRA-12786)
 + * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 +Merged from 2.1:
+  * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
   * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
 - * Fix missed signal when commit log segments are recycled (CASSANDRA-13037)
 - * Fix RecoveryManagerTruncateTest (CASSANDRA-12802)
   * Don't skip sstables based on maxLocalDeletionTime (CASSANDRA-12765)
  
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2acc15ba/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index c5f34ae,905a981..efa9330
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@@ -81,10 -81,11 +81,12 @@@ public class CustomTThreadPoolServer ex
      public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService) {
          super(args);
          this.executorService = executorService;
+         this.stopped = false;
          this.args = args;
+         this.activeClients = new AtomicInteger(0);
      }
  
 +    @SuppressWarnings("resource")
      public void serve()
      {
          try


[23/30] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by st...@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/6ff56c30
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6ff56c30
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6ff56c30

Branch: refs/heads/cassandra-3.11
Commit: 6ff56c30e0913b49291a5a61a49eacac8ee3eb28
Parents: d8eec1c 2acc15b
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jan 11 07:45:27 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:47:06 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 3 ++-
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6ff56c30/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index a533e6f,a7e89ed..5f37cd7
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,25 -1,4 +1,26 @@@
 -2.2.9
 +3.0.11
 + * Replace empty strings with null values if they cannot be converted (CASSANDRA-12794)
 + * Fixed flacky SSTableRewriterTest: check file counts before calling validateCFS (CASSANDRA-12348)
 + * Fix deserialization of 2.x DeletedCells (CASSANDRA-12620)
 + * Add parent repair session id to anticompaction log message (CASSANDRA-12186)
 + * Improve contention handling on failure to acquire MV lock for streaming and hints (CASSANDRA-12905)
 + * Fix DELETE and UPDATE queries with empty IN restrictions (CASSANDRA-12829)
 + * Mark MVs as built after successful bootstrap (CASSANDRA-12984)
 + * Estimated TS drop-time histogram updated with Cell.NO_DELETION_TIME (CASSANDRA-13040)
 + * Nodetool compactionstats fails with NullPointerException (CASSANDRA-13021)
 + * Thread local pools never cleaned up (CASSANDRA-13033)
 + * Set RPC_READY to false when draining or if a node is marked as shutdown (CASSANDRA-12781)
 + * CQL often queries static columns unnecessarily (CASSANDRA-12768)
 + * Make sure sstables only get committed when it's safe to discard commit log records (CASSANDRA-12956)
 + * Reject default_time_to_live option when creating or altering MVs (CASSANDRA-12868)
 + * Nodetool should use a more sane max heap size (CASSANDRA-12739)
 + * LocalToken ensures token values are cloned on heap (CASSANDRA-12651)
 + * AnticompactionRequestSerializer serializedSize is incorrect (CASSANDRA-12934)
 + * Prevent reloading of logback.xml from UDF sandbox (CASSANDRA-12535)
 + * Reenable HeapPool (CASSANDRA-12900)
-  * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 +Merged from 2.2:
++ * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
++ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
   * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
   * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
   * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)


[19/30] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by st...@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/2acc15ba
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2acc15ba
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2acc15ba

Branch: refs/heads/cassandra-2.2
Commit: 2acc15ba38695495acdb678dcb005d7f611bae2b
Parents: dbe039f 47f7d9c
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jan 11 07:37:22 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:37:22 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 1 +
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2acc15ba/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 4b301ee,23fd12a..a7e89ed
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,32 -1,8 +1,33 @@@
 -2.1.17
 +2.2.9
 + * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
 + * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
 + * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)
 + * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
 + * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
 + * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
 + * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)
 + * Use saved tokens when setting local tokens on StorageService.joinRing (CASSANDRA-12935)
 + * cqlsh: fix DESC TYPES errors (CASSANDRA-12914)
 + * Fix leak on skipped SSTables in sstableupgrade (CASSANDRA-12899)
 + * Avoid blocking gossip during pending range calculation (CASSANDRA-12281)
 + * Fix purgeability of tombstones with max timestamp (CASSANDRA-12792)
 + * Fail repair if participant dies during sync or anticompaction (CASSANDRA-12901)
 + * cqlsh COPY: unprotected pk values before converting them if not using prepared statements (CASSANDRA-12863)
 + * Fix Util.spinAssertEquals (CASSANDRA-12283)
 + * Fix potential NPE for compactionstats (CASSANDRA-12462)
 + * Prepare legacy authenticate statement if credentials table initialised after node startup (CASSANDRA-12813)
 + * Change cassandra.wait_for_tracing_events_timeout_secs default to 0 (CASSANDRA-12754)
 + * Clean up permissions when a UDA is dropped (CASSANDRA-12720)
 + * Limit colUpdateTimeDelta histogram updates to reasonable deltas (CASSANDRA-11117)
 + * Fix leak errors and execution rejected exceptions when draining (CASSANDRA-12457)
 + * Fix merkle tree depth calculation (CASSANDRA-12580)
 + * Make Collections deserialization more robust (CASSANDRA-12618)
 + * Better handle invalid system roles table (CASSANDRA-12700)
 + * Split consistent range movement flag correction (CASSANDRA-12786)
 + * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 +Merged from 2.1:
+  * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
   * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
 - * Fix missed signal when commit log segments are recycled (CASSANDRA-13037)
 - * Fix RecoveryManagerTruncateTest (CASSANDRA-12802)
   * Don't skip sstables based on maxLocalDeletionTime (CASSANDRA-12765)
  
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2acc15ba/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index c5f34ae,905a981..efa9330
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@@ -81,10 -81,11 +81,12 @@@ public class CustomTThreadPoolServer ex
      public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService) {
          super(args);
          this.executorService = executorService;
+         this.stopped = false;
          this.args = args;
+         this.activeClients = new AtomicInteger(0);
      }
  
 +    @SuppressWarnings("resource")
      public void serve()
      {
          try


[11/30] cassandra git commit: Fix race causing infinite loop if Thrift server is stopped before it starts listening

Posted by st...@apache.org.
Fix race causing infinite loop if Thrift server is stopped before it starts listening

patch by Stefania Alborghetti; reviewed by Joel Knighton for CASSANDRA-12856


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

Branch: refs/heads/cassandra-2.1
Commit: 47f7d9c6aab694bf956fa2174fe0dd36702e01a6
Parents: bdca25e
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Tue Jan 3 10:09:52 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:34:15 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 1 +
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/47f7d9c6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a99f79a..23fd12a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.17
+ * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
  * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
  * Fix missed signal when commit log segments are recycled (CASSANDRA-13037)
  * Fix RecoveryManagerTruncateTest (CASSANDRA-12802)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/47f7d9c6/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index 7e67110..905a981 100644
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@ -75,13 +75,15 @@ public class CustomTThreadPoolServer extends TServer
     private final TThreadPoolServer.Args args;
 
     //Track and Limit the number of connected clients
-    private final AtomicInteger activeClients = new AtomicInteger(0);
+    private final AtomicInteger activeClients;
 
 
     public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService) {
         super(args);
         this.executorService = executorService;
+        this.stopped = false;
         this.args = args;
+        this.activeClients = new AtomicInteger(0);
     }
 
     public void serve()
@@ -96,7 +98,6 @@ public class CustomTThreadPoolServer extends TServer
             return;
         }
 
-        stopped = false;
         while (!stopped)
         {
             // block until we are under max clients


[18/30] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by st...@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/2acc15ba
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2acc15ba
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2acc15ba

Branch: refs/heads/cassandra-3.0
Commit: 2acc15ba38695495acdb678dcb005d7f611bae2b
Parents: dbe039f 47f7d9c
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jan 11 07:37:22 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:37:22 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 1 +
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2acc15ba/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 4b301ee,23fd12a..a7e89ed
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,32 -1,8 +1,33 @@@
 -2.1.17
 +2.2.9
 + * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
 + * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
 + * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)
 + * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
 + * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
 + * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
 + * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)
 + * Use saved tokens when setting local tokens on StorageService.joinRing (CASSANDRA-12935)
 + * cqlsh: fix DESC TYPES errors (CASSANDRA-12914)
 + * Fix leak on skipped SSTables in sstableupgrade (CASSANDRA-12899)
 + * Avoid blocking gossip during pending range calculation (CASSANDRA-12281)
 + * Fix purgeability of tombstones with max timestamp (CASSANDRA-12792)
 + * Fail repair if participant dies during sync or anticompaction (CASSANDRA-12901)
 + * cqlsh COPY: unprotected pk values before converting them if not using prepared statements (CASSANDRA-12863)
 + * Fix Util.spinAssertEquals (CASSANDRA-12283)
 + * Fix potential NPE for compactionstats (CASSANDRA-12462)
 + * Prepare legacy authenticate statement if credentials table initialised after node startup (CASSANDRA-12813)
 + * Change cassandra.wait_for_tracing_events_timeout_secs default to 0 (CASSANDRA-12754)
 + * Clean up permissions when a UDA is dropped (CASSANDRA-12720)
 + * Limit colUpdateTimeDelta histogram updates to reasonable deltas (CASSANDRA-11117)
 + * Fix leak errors and execution rejected exceptions when draining (CASSANDRA-12457)
 + * Fix merkle tree depth calculation (CASSANDRA-12580)
 + * Make Collections deserialization more robust (CASSANDRA-12618)
 + * Better handle invalid system roles table (CASSANDRA-12700)
 + * Split consistent range movement flag correction (CASSANDRA-12786)
 + * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 +Merged from 2.1:
+  * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
   * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
 - * Fix missed signal when commit log segments are recycled (CASSANDRA-13037)
 - * Fix RecoveryManagerTruncateTest (CASSANDRA-12802)
   * Don't skip sstables based on maxLocalDeletionTime (CASSANDRA-12765)
  
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2acc15ba/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index c5f34ae,905a981..efa9330
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@@ -81,10 -81,11 +81,12 @@@ public class CustomTThreadPoolServer ex
      public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService) {
          super(args);
          this.executorService = executorService;
+         this.stopped = false;
          this.args = args;
+         this.activeClients = new AtomicInteger(0);
      }
  
 +    @SuppressWarnings("resource")
      public void serve()
      {
          try


[04/30] cassandra git commit: Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

Posted by st...@apache.org.
Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

patch by Jon Haddad; reviewed by Nate McCall for CASSANDRA-12979


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

Branch: refs/heads/cassandra-3.X
Commit: dbe039f628220e5141836ecb9aab05d4da60589a
Parents: 6f360b6
Author: Jon Haddad <jo...@jonhaddad.com>
Authored: Wed Jan 4 11:52:54 2017 -0800
Committer: Nate McCall <zz...@gmail.com>
Committed: Tue Jan 10 15:15:47 2017 +1300

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  6 +++
 .../cassandra/db/compaction/CompactionTask.java | 40 +++++++++++++++-----
 3 files changed, 38 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbe039f6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b41313d..4b301ee 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -24,6 +24,7 @@
  * Make Collections deserialization more robust (CASSANDRA-12618)
  * Better handle invalid system roles table (CASSANDRA-12700)
  * Split consistent range movement flag correction (CASSANDRA-12786)
+ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 Merged from 2.1:
  * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
  * Don't skip sstables based on maxLocalDeletionTime (CASSANDRA-12765)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbe039f6/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 37949a1..06e9eeb 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -16,6 +16,12 @@ using the provided 'sstableupgrade' tool.
 2.2.9
 =====
 
+Upgrading
+---------
+   - Compaction now correctly drops sstables out of CompactionTask when there
+     isn't enough disk space to perform the full compaction.  This should reduce
+     pending compaction tasks on systems with little remaining disk space.
+
 Deprecation
 -----------
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbe039f6/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 20d3dc0..7489b3d 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -75,12 +75,14 @@ public class CompactionTask extends AbstractCompactionTask
         return transaction.originals().size();
     }
 
-    public boolean reduceScopeForLimitedSpace()
+    public boolean reduceScopeForLimitedSpace(long expectedSize)
     {
         if (partialCompactionsAcceptable() && transaction.originals().size() > 1)
         {
             // Try again w/o the largest one.
-            logger.warn("insufficient space to compact all requested files {}", StringUtils.join(transaction.originals(), ", "));
+            logger.warn("insufficient space to compact all requested files. {}MB required, {}",
+                        (float) expectedSize / 1024 / 1024,
+                        StringUtils.join(transaction.originals(), ", "));
             // Note that we have removed files that are still marked as compacting.
             // This suboptimal but ok since the caller will unmark all the sstables at the end.
             SSTableReader removedSSTable = cfs.getMaxSizeFile(transaction.originals());
@@ -113,9 +115,8 @@ public class CompactionTask extends AbstractCompactionTask
 
         // note that we need to do a rough estimate early if we can fit the compaction on disk - this is pessimistic, but
         // since we might remove sstables from the compaction in checkAvailableDiskSpace it needs to be done here
-        long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
-        long earlySSTableEstimate = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
-        checkAvailableDiskSpace(earlySSTableEstimate, expectedWriteSize);
+
+        checkAvailableDiskSpace();
 
         // sanity check: all sstables must belong to the same cfs
         assert !Iterables.any(transaction.originals(), new Predicate<SSTableReader>()
@@ -271,12 +272,33 @@ public class CompactionTask extends AbstractCompactionTask
         return minRepairedAt;
     }
 
-    protected void checkAvailableDiskSpace(long estimatedSSTables, long expectedWriteSize)
+    /*
+    Checks if we have enough disk space to execute the compaction.  Drops the largest sstable out of the Task until
+    there's enough space (in theory) to handle the compaction.  Does not take into account space that will be taken by
+    other compactions.
+     */
+    protected void checkAvailableDiskSpace()
     {
-        while (!cfs.directories.hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+        AbstractCompactionStrategy strategy = cfs.getCompactionStrategy();
+
+        while(true)
         {
-            if (!reduceScopeForLimitedSpace())
-                throw new RuntimeException(String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize));
+            long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
+            long estimatedSSTables = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
+
+            if(cfs.directories.hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+                break;
+
+            if (!reduceScopeForLimitedSpace(expectedWriteSize))
+            {
+                // we end up here if we can't take any more sstables out of the compaction.
+                // usually means we've run out of disk space
+                String msg = String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize);
+                logger.warn(msg);
+                throw new RuntimeException(msg);
+            }
+            logger.warn("Not enough space for compaction, {}MB estimated.  Reducing scope.",
+                            (float) expectedWriteSize / 1024 / 1024);
         }
     }
 


[08/30] cassandra git commit: Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

Posted by st...@apache.org.
Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

patch by Jon Haddad; reviewed by Nate McCall for CASSANDRA-12979


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

Branch: refs/heads/trunk
Commit: e9251e1fbc8619db92ee3818eb9e8abc9a0fade5
Parents: 858cb25
Author: Jon Haddad <jo...@jonhaddad.com>
Authored: Wed Jan 4 11:52:54 2017 -0800
Committer: Nate McCall <zz...@gmail.com>
Committed: Tue Jan 10 16:24:37 2017 +1300

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  3 ++
 .../cassandra/db/compaction/CompactionTask.java | 41 +++++++++++++++-----
 3 files changed, 36 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e9251e1f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 1b74dc7..99f6613 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -110,6 +110,7 @@
  * Restore resumable hints delivery (CASSANDRA-11960)
  * Properly report LWT contention (CASSANDRA-12626)
 Merged from 3.0:
+ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
  * Replace empty strings with null values if they cannot be converted (CASSANDRA-12794)
  * Fix deserialization of 2.x DeletedCells (CASSANDRA-12620)
  * Add parent repair session id to anticompaction log message (CASSANDRA-12186)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e9251e1f/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 9f376ff..7bccaae 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -24,6 +24,9 @@ Upgrading
    - Only Java and JavaScript are now supported UDF languages.
      The sandbox in 3.0 already prevented the use of script languages except Java
      and JavaScript.
+   - Compaction now correctly drops sstables out of CompactionTask when there
+     isn't enough disk space to perform the full compaction.  This should reduce
+     pending compaction tasks on systems with little remaining disk space.
 
 3.10
 ====

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e9251e1f/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index a9d6c7c..b2e9b8c 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -86,12 +86,15 @@ public class CompactionTask extends AbstractCompactionTask
         return transaction.originals().size();
     }
 
-    public boolean reduceScopeForLimitedSpace()
+    public boolean reduceScopeForLimitedSpace(long expectedSize)
     {
         if (partialCompactionsAcceptable() && transaction.originals().size() > 1)
         {
             // Try again w/o the largest one.
-            logger.warn("Insufficient space to compact all requested files {}", StringUtils.join(transaction.originals(), ", "));
+            logger.warn("insufficient space to compact all requested files. {}MB required, {}",
+                        (float) expectedSize / 1024 / 1024,
+                        StringUtils.join(transaction.originals(), ", "));
+
             // Note that we have removed files that are still marked as compacting.
             // This suboptimal but ok since the caller will unmark all the sstables at the end.
             SSTableReader removedSSTable = cfs.getMaxSizeFile(transaction.originals());
@@ -124,9 +127,8 @@ public class CompactionTask extends AbstractCompactionTask
 
         // note that we need to do a rough estimate early if we can fit the compaction on disk - this is pessimistic, but
         // since we might remove sstables from the compaction in checkAvailableDiskSpace it needs to be done here
-        long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
-        long earlySSTableEstimate = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
-        checkAvailableDiskSpace(earlySSTableEstimate, expectedWriteSize);
+
+        checkAvailableDiskSpace();
 
         // sanity check: all sstables must belong to the same cfs
         assert !Iterables.any(transaction.originals(), new Predicate<SSTableReader>()
@@ -317,7 +319,12 @@ public class CompactionTask extends AbstractCompactionTask
         return minRepairedAt;
     }
 
-    protected void checkAvailableDiskSpace(long estimatedSSTables, long expectedWriteSize)
+    /*
+    Checks if we have enough disk space to execute the compaction.  Drops the largest sstable out of the Task until
+    there's enough space (in theory) to handle the compaction.  Does not take into account space that will be taken by
+    other compactions.
+     */
+    protected void checkAvailableDiskSpace()
     {
         if(!cfs.isCompactionDiskSpaceCheckEnabled() && compactionType == OperationType.COMPACTION)
         {
@@ -325,10 +332,26 @@ public class CompactionTask extends AbstractCompactionTask
             return;
         }
 
-        while (!getDirectories().hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+        CompactionStrategyManager strategy = cfs.getCompactionStrategyManager();
+
+        while(true)
         {
-            if (!reduceScopeForLimitedSpace())
-                throw new RuntimeException(String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize));
+            long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
+            long estimatedSSTables = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
+
+            if(cfs.getDirectories().hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+                break;
+
+            if (!reduceScopeForLimitedSpace(expectedWriteSize))
+            {
+                // we end up here if we can't take any more sstables out of the compaction.
+                // usually means we've run out of disk space
+                String msg = String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize);
+                logger.warn(msg);
+                throw new RuntimeException(msg);
+            }
+            logger.warn("Not enough space for compaction, {}MB estimated.  Reducing scope.",
+                            (float) expectedWriteSize / 1024 / 1024);
         }
     }
 


[15/30] cassandra git commit: Fix race causing infinite loop if Thrift server is stopped before it starts listening

Posted by st...@apache.org.
Fix race causing infinite loop if Thrift server is stopped before it starts listening

patch by Stefania Alborghetti; reviewed by Joel Knighton for CASSANDRA-12856


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

Branch: refs/heads/cassandra-3.X
Commit: 47f7d9c6aab694bf956fa2174fe0dd36702e01a6
Parents: bdca25e
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Tue Jan 3 10:09:52 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:34:15 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 1 +
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/47f7d9c6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a99f79a..23fd12a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.17
+ * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
  * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
  * Fix missed signal when commit log segments are recycled (CASSANDRA-13037)
  * Fix RecoveryManagerTruncateTest (CASSANDRA-12802)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/47f7d9c6/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index 7e67110..905a981 100644
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@ -75,13 +75,15 @@ public class CustomTThreadPoolServer extends TServer
     private final TThreadPoolServer.Args args;
 
     //Track and Limit the number of connected clients
-    private final AtomicInteger activeClients = new AtomicInteger(0);
+    private final AtomicInteger activeClients;
 
 
     public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService) {
         super(args);
         this.executorService = executorService;
+        this.stopped = false;
         this.args = args;
+        this.activeClients = new AtomicInteger(0);
     }
 
     public void serve()
@@ -96,7 +98,6 @@ public class CustomTThreadPoolServer extends TServer
             return;
         }
 
-        stopped = false;
         while (!stopped)
         {
             // block until we are under max clients


[28/30] cassandra git commit: Merge branch 'cassandra-3.11' into cassandra-3.X

Posted by st...@apache.org.
Merge branch 'cassandra-3.11' into cassandra-3.X


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

Branch: refs/heads/cassandra-3.X
Commit: 699eff0716705dd27fc45443bcd6b5e3797314f7
Parents: d1d75ad b52c820
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jan 11 07:57:42 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:57:42 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                   | 7 ++++---
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java  | 5 +++--
 2 files changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/699eff07/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index ba4172b,b8250ab..917eb5d
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,26 -1,3 +1,24 @@@
 +3.12
 + * Use new token allocation for non bootstrap case as well (CASSANDRA-13080)
 + * Avoid byte-array copy when key cache is disabled (CASSANDRA-13084)
 + * More fixes to the TokenAllocator (CASSANDRA-12990)
 + * Require forceful decommission if number of nodes is less than replication factor (CASSANDRA-12510)
 + * Allow IN restrictions on column families with collections (CASSANDRA-12654)
 + * Move to FastThreadLocalThread and FastThreadLocal (CASSANDRA-13034)
 + * nodetool stopdaemon errors out (CASSANDRA-13030)
 + * Log message size in trace message in OutboundTcpConnection (CASSANDRA-13028)
 + * Add timeUnit Days for cassandra-stress (CASSANDRA-13029)
 + * Add mutation size and batch metrics (CASSANDRA-12649)
 + * Add method to get size of endpoints to TokenMetadata (CASSANDRA-12999)
 + * Fix primary index calculation for SASI (CASSANDRA-12910)
 + * Expose time spent waiting in thread pool queue (CASSANDRA-8398)
 + * Conditionally update index built status to avoid unnecessary flushes (CASSANDRA-12969)
 + * NoReplicationTokenAllocator should work with zero replication factor (CASSANDRA-12983)
 + * cqlsh auto completion: refactor definition of compaction strategy options (CASSANDRA-12946)
 + * Add support for arithmetic operators (CASSANDRA-11935)
 + * Tables in system_distributed should not use gcgs of 0 (CASSANDRA-12954)
- Merged from 3.0:
-  * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
-  * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
++
 +
  3.10
   * Fixed query monitoring for range queries (CASSANDRA-13050)
   * Remove outboundBindAny configuration property (CASSANDRA-12673)
@@@ -134,6 -111,6 +132,7 @@@
   * Properly report LWT contention (CASSANDRA-12626)
  Merged from 3.0:
   * Replace empty strings with null values if they cannot be converted (CASSANDRA-12794)
++ * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
   * Fix deserialization of 2.x DeletedCells (CASSANDRA-12620)
   * Add parent repair session id to anticompaction log message (CASSANDRA-12186)
   * Improve contention handling on failure to acquire MV lock for streaming and hints (CASSANDRA-12905)
@@@ -199,6 -176,9 +198,8 @@@
   * Correct log message for statistics of offheap memtable flush (CASSANDRA-12776)
   * Explicitly set locale for string validation (CASSANDRA-12541,CASSANDRA-12542,CASSANDRA-12543,CASSANDRA-12545)
  Merged from 2.2:
+  * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
+  * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 - * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
   * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
   * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)
   * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)


[13/30] cassandra git commit: Fix race causing infinite loop if Thrift server is stopped before it starts listening

Posted by st...@apache.org.
Fix race causing infinite loop if Thrift server is stopped before it starts listening

patch by Stefania Alborghetti; reviewed by Joel Knighton for CASSANDRA-12856


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

Branch: refs/heads/cassandra-3.0
Commit: 47f7d9c6aab694bf956fa2174fe0dd36702e01a6
Parents: bdca25e
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Tue Jan 3 10:09:52 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:34:15 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 1 +
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/47f7d9c6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a99f79a..23fd12a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.17
+ * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
  * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
  * Fix missed signal when commit log segments are recycled (CASSANDRA-13037)
  * Fix RecoveryManagerTruncateTest (CASSANDRA-12802)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/47f7d9c6/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index 7e67110..905a981 100644
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@ -75,13 +75,15 @@ public class CustomTThreadPoolServer extends TServer
     private final TThreadPoolServer.Args args;
 
     //Track and Limit the number of connected clients
-    private final AtomicInteger activeClients = new AtomicInteger(0);
+    private final AtomicInteger activeClients;
 
 
     public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService) {
         super(args);
         this.executorService = executorService;
+        this.stopped = false;
         this.args = args;
+        this.activeClients = new AtomicInteger(0);
     }
 
     public void serve()
@@ -96,7 +98,6 @@ public class CustomTThreadPoolServer extends TServer
             return;
         }
 
-        stopped = false;
         while (!stopped)
         {
             // block until we are under max clients


[12/30] cassandra git commit: Fix race causing infinite loop if Thrift server is stopped before it starts listening

Posted by st...@apache.org.
Fix race causing infinite loop if Thrift server is stopped before it starts listening

patch by Stefania Alborghetti; reviewed by Joel Knighton for CASSANDRA-12856


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

Branch: refs/heads/trunk
Commit: 47f7d9c6aab694bf956fa2174fe0dd36702e01a6
Parents: bdca25e
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Tue Jan 3 10:09:52 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:34:15 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 1 +
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/47f7d9c6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a99f79a..23fd12a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.17
+ * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
  * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
  * Fix missed signal when commit log segments are recycled (CASSANDRA-13037)
  * Fix RecoveryManagerTruncateTest (CASSANDRA-12802)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/47f7d9c6/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index 7e67110..905a981 100644
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@ -75,13 +75,15 @@ public class CustomTThreadPoolServer extends TServer
     private final TThreadPoolServer.Args args;
 
     //Track and Limit the number of connected clients
-    private final AtomicInteger activeClients = new AtomicInteger(0);
+    private final AtomicInteger activeClients;
 
 
     public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService) {
         super(args);
         this.executorService = executorService;
+        this.stopped = false;
         this.args = args;
+        this.activeClients = new AtomicInteger(0);
     }
 
     public void serve()
@@ -96,7 +98,6 @@ public class CustomTThreadPoolServer extends TServer
             return;
         }
 
-        stopped = false;
         while (!stopped)
         {
             // block until we are under max clients


[17/30] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by st...@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/2acc15ba
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2acc15ba
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2acc15ba

Branch: refs/heads/trunk
Commit: 2acc15ba38695495acdb678dcb005d7f611bae2b
Parents: dbe039f 47f7d9c
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jan 11 07:37:22 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:37:22 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 1 +
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2acc15ba/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 4b301ee,23fd12a..a7e89ed
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,32 -1,8 +1,33 @@@
 -2.1.17
 +2.2.9
 + * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
 + * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
 + * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)
 + * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
 + * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
 + * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
 + * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)
 + * Use saved tokens when setting local tokens on StorageService.joinRing (CASSANDRA-12935)
 + * cqlsh: fix DESC TYPES errors (CASSANDRA-12914)
 + * Fix leak on skipped SSTables in sstableupgrade (CASSANDRA-12899)
 + * Avoid blocking gossip during pending range calculation (CASSANDRA-12281)
 + * Fix purgeability of tombstones with max timestamp (CASSANDRA-12792)
 + * Fail repair if participant dies during sync or anticompaction (CASSANDRA-12901)
 + * cqlsh COPY: unprotected pk values before converting them if not using prepared statements (CASSANDRA-12863)
 + * Fix Util.spinAssertEquals (CASSANDRA-12283)
 + * Fix potential NPE for compactionstats (CASSANDRA-12462)
 + * Prepare legacy authenticate statement if credentials table initialised after node startup (CASSANDRA-12813)
 + * Change cassandra.wait_for_tracing_events_timeout_secs default to 0 (CASSANDRA-12754)
 + * Clean up permissions when a UDA is dropped (CASSANDRA-12720)
 + * Limit colUpdateTimeDelta histogram updates to reasonable deltas (CASSANDRA-11117)
 + * Fix leak errors and execution rejected exceptions when draining (CASSANDRA-12457)
 + * Fix merkle tree depth calculation (CASSANDRA-12580)
 + * Make Collections deserialization more robust (CASSANDRA-12618)
 + * Better handle invalid system roles table (CASSANDRA-12700)
 + * Split consistent range movement flag correction (CASSANDRA-12786)
 + * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 +Merged from 2.1:
+  * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
   * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
 - * Fix missed signal when commit log segments are recycled (CASSANDRA-13037)
 - * Fix RecoveryManagerTruncateTest (CASSANDRA-12802)
   * Don't skip sstables based on maxLocalDeletionTime (CASSANDRA-12765)
  
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2acc15ba/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index c5f34ae,905a981..efa9330
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@@ -81,10 -81,11 +81,12 @@@ public class CustomTThreadPoolServer ex
      public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService) {
          super(args);
          this.executorService = executorService;
+         this.stopped = false;
          this.args = args;
+         this.activeClients = new AtomicInteger(0);
      }
  
 +    @SuppressWarnings("resource")
      public void serve()
      {
          try


[02/30] cassandra git commit: Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

Posted by st...@apache.org.
Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

patch by Jon Haddad; reviewed by Nate McCall for CASSANDRA-12979


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

Branch: refs/heads/cassandra-3.11
Commit: dbe039f628220e5141836ecb9aab05d4da60589a
Parents: 6f360b6
Author: Jon Haddad <jo...@jonhaddad.com>
Authored: Wed Jan 4 11:52:54 2017 -0800
Committer: Nate McCall <zz...@gmail.com>
Committed: Tue Jan 10 15:15:47 2017 +1300

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  6 +++
 .../cassandra/db/compaction/CompactionTask.java | 40 +++++++++++++++-----
 3 files changed, 38 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbe039f6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b41313d..4b301ee 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -24,6 +24,7 @@
  * Make Collections deserialization more robust (CASSANDRA-12618)
  * Better handle invalid system roles table (CASSANDRA-12700)
  * Split consistent range movement flag correction (CASSANDRA-12786)
+ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 Merged from 2.1:
  * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
  * Don't skip sstables based on maxLocalDeletionTime (CASSANDRA-12765)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbe039f6/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 37949a1..06e9eeb 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -16,6 +16,12 @@ using the provided 'sstableupgrade' tool.
 2.2.9
 =====
 
+Upgrading
+---------
+   - Compaction now correctly drops sstables out of CompactionTask when there
+     isn't enough disk space to perform the full compaction.  This should reduce
+     pending compaction tasks on systems with little remaining disk space.
+
 Deprecation
 -----------
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbe039f6/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 20d3dc0..7489b3d 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -75,12 +75,14 @@ public class CompactionTask extends AbstractCompactionTask
         return transaction.originals().size();
     }
 
-    public boolean reduceScopeForLimitedSpace()
+    public boolean reduceScopeForLimitedSpace(long expectedSize)
     {
         if (partialCompactionsAcceptable() && transaction.originals().size() > 1)
         {
             // Try again w/o the largest one.
-            logger.warn("insufficient space to compact all requested files {}", StringUtils.join(transaction.originals(), ", "));
+            logger.warn("insufficient space to compact all requested files. {}MB required, {}",
+                        (float) expectedSize / 1024 / 1024,
+                        StringUtils.join(transaction.originals(), ", "));
             // Note that we have removed files that are still marked as compacting.
             // This suboptimal but ok since the caller will unmark all the sstables at the end.
             SSTableReader removedSSTable = cfs.getMaxSizeFile(transaction.originals());
@@ -113,9 +115,8 @@ public class CompactionTask extends AbstractCompactionTask
 
         // note that we need to do a rough estimate early if we can fit the compaction on disk - this is pessimistic, but
         // since we might remove sstables from the compaction in checkAvailableDiskSpace it needs to be done here
-        long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
-        long earlySSTableEstimate = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
-        checkAvailableDiskSpace(earlySSTableEstimate, expectedWriteSize);
+
+        checkAvailableDiskSpace();
 
         // sanity check: all sstables must belong to the same cfs
         assert !Iterables.any(transaction.originals(), new Predicate<SSTableReader>()
@@ -271,12 +272,33 @@ public class CompactionTask extends AbstractCompactionTask
         return minRepairedAt;
     }
 
-    protected void checkAvailableDiskSpace(long estimatedSSTables, long expectedWriteSize)
+    /*
+    Checks if we have enough disk space to execute the compaction.  Drops the largest sstable out of the Task until
+    there's enough space (in theory) to handle the compaction.  Does not take into account space that will be taken by
+    other compactions.
+     */
+    protected void checkAvailableDiskSpace()
     {
-        while (!cfs.directories.hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+        AbstractCompactionStrategy strategy = cfs.getCompactionStrategy();
+
+        while(true)
         {
-            if (!reduceScopeForLimitedSpace())
-                throw new RuntimeException(String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize));
+            long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
+            long estimatedSSTables = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
+
+            if(cfs.directories.hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+                break;
+
+            if (!reduceScopeForLimitedSpace(expectedWriteSize))
+            {
+                // we end up here if we can't take any more sstables out of the compaction.
+                // usually means we've run out of disk space
+                String msg = String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize);
+                logger.warn(msg);
+                throw new RuntimeException(msg);
+            }
+            logger.warn("Not enough space for compaction, {}MB estimated.  Reducing scope.",
+                            (float) expectedWriteSize / 1024 / 1024);
         }
     }
 


[29/30] cassandra git commit: Merge branch 'cassandra-3.11' into cassandra-3.X

Posted by st...@apache.org.
Merge branch 'cassandra-3.11' into cassandra-3.X


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

Branch: refs/heads/trunk
Commit: 699eff0716705dd27fc45443bcd6b5e3797314f7
Parents: d1d75ad b52c820
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jan 11 07:57:42 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:57:42 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                   | 7 ++++---
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java  | 5 +++--
 2 files changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/699eff07/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index ba4172b,b8250ab..917eb5d
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,26 -1,3 +1,24 @@@
 +3.12
 + * Use new token allocation for non bootstrap case as well (CASSANDRA-13080)
 + * Avoid byte-array copy when key cache is disabled (CASSANDRA-13084)
 + * More fixes to the TokenAllocator (CASSANDRA-12990)
 + * Require forceful decommission if number of nodes is less than replication factor (CASSANDRA-12510)
 + * Allow IN restrictions on column families with collections (CASSANDRA-12654)
 + * Move to FastThreadLocalThread and FastThreadLocal (CASSANDRA-13034)
 + * nodetool stopdaemon errors out (CASSANDRA-13030)
 + * Log message size in trace message in OutboundTcpConnection (CASSANDRA-13028)
 + * Add timeUnit Days for cassandra-stress (CASSANDRA-13029)
 + * Add mutation size and batch metrics (CASSANDRA-12649)
 + * Add method to get size of endpoints to TokenMetadata (CASSANDRA-12999)
 + * Fix primary index calculation for SASI (CASSANDRA-12910)
 + * Expose time spent waiting in thread pool queue (CASSANDRA-8398)
 + * Conditionally update index built status to avoid unnecessary flushes (CASSANDRA-12969)
 + * NoReplicationTokenAllocator should work with zero replication factor (CASSANDRA-12983)
 + * cqlsh auto completion: refactor definition of compaction strategy options (CASSANDRA-12946)
 + * Add support for arithmetic operators (CASSANDRA-11935)
 + * Tables in system_distributed should not use gcgs of 0 (CASSANDRA-12954)
- Merged from 3.0:
-  * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
-  * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
++
 +
  3.10
   * Fixed query monitoring for range queries (CASSANDRA-13050)
   * Remove outboundBindAny configuration property (CASSANDRA-12673)
@@@ -134,6 -111,6 +132,7 @@@
   * Properly report LWT contention (CASSANDRA-12626)
  Merged from 3.0:
   * Replace empty strings with null values if they cannot be converted (CASSANDRA-12794)
++ * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
   * Fix deserialization of 2.x DeletedCells (CASSANDRA-12620)
   * Add parent repair session id to anticompaction log message (CASSANDRA-12186)
   * Improve contention handling on failure to acquire MV lock for streaming and hints (CASSANDRA-12905)
@@@ -199,6 -176,9 +198,8 @@@
   * Correct log message for statistics of offheap memtable flush (CASSANDRA-12776)
   * Explicitly set locale for string validation (CASSANDRA-12541,CASSANDRA-12542,CASSANDRA-12543,CASSANDRA-12545)
  Merged from 2.2:
+  * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
+  * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 - * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
   * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
   * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)
   * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)


[21/30] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by st...@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/6ff56c30
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6ff56c30
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6ff56c30

Branch: refs/heads/trunk
Commit: 6ff56c30e0913b49291a5a61a49eacac8ee3eb28
Parents: d8eec1c 2acc15b
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jan 11 07:45:27 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:47:06 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 3 ++-
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6ff56c30/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index a533e6f,a7e89ed..5f37cd7
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,25 -1,4 +1,26 @@@
 -2.2.9
 +3.0.11
 + * Replace empty strings with null values if they cannot be converted (CASSANDRA-12794)
 + * Fixed flacky SSTableRewriterTest: check file counts before calling validateCFS (CASSANDRA-12348)
 + * Fix deserialization of 2.x DeletedCells (CASSANDRA-12620)
 + * Add parent repair session id to anticompaction log message (CASSANDRA-12186)
 + * Improve contention handling on failure to acquire MV lock for streaming and hints (CASSANDRA-12905)
 + * Fix DELETE and UPDATE queries with empty IN restrictions (CASSANDRA-12829)
 + * Mark MVs as built after successful bootstrap (CASSANDRA-12984)
 + * Estimated TS drop-time histogram updated with Cell.NO_DELETION_TIME (CASSANDRA-13040)
 + * Nodetool compactionstats fails with NullPointerException (CASSANDRA-13021)
 + * Thread local pools never cleaned up (CASSANDRA-13033)
 + * Set RPC_READY to false when draining or if a node is marked as shutdown (CASSANDRA-12781)
 + * CQL often queries static columns unnecessarily (CASSANDRA-12768)
 + * Make sure sstables only get committed when it's safe to discard commit log records (CASSANDRA-12956)
 + * Reject default_time_to_live option when creating or altering MVs (CASSANDRA-12868)
 + * Nodetool should use a more sane max heap size (CASSANDRA-12739)
 + * LocalToken ensures token values are cloned on heap (CASSANDRA-12651)
 + * AnticompactionRequestSerializer serializedSize is incorrect (CASSANDRA-12934)
 + * Prevent reloading of logback.xml from UDF sandbox (CASSANDRA-12535)
 + * Reenable HeapPool (CASSANDRA-12900)
-  * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 +Merged from 2.2:
++ * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
++ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
   * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
   * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
   * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)


[24/30] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by st...@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/6ff56c30
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6ff56c30
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6ff56c30

Branch: refs/heads/cassandra-3.0
Commit: 6ff56c30e0913b49291a5a61a49eacac8ee3eb28
Parents: d8eec1c 2acc15b
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jan 11 07:45:27 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:47:06 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 3 ++-
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6ff56c30/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index a533e6f,a7e89ed..5f37cd7
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,25 -1,4 +1,26 @@@
 -2.2.9
 +3.0.11
 + * Replace empty strings with null values if they cannot be converted (CASSANDRA-12794)
 + * Fixed flacky SSTableRewriterTest: check file counts before calling validateCFS (CASSANDRA-12348)
 + * Fix deserialization of 2.x DeletedCells (CASSANDRA-12620)
 + * Add parent repair session id to anticompaction log message (CASSANDRA-12186)
 + * Improve contention handling on failure to acquire MV lock for streaming and hints (CASSANDRA-12905)
 + * Fix DELETE and UPDATE queries with empty IN restrictions (CASSANDRA-12829)
 + * Mark MVs as built after successful bootstrap (CASSANDRA-12984)
 + * Estimated TS drop-time histogram updated with Cell.NO_DELETION_TIME (CASSANDRA-13040)
 + * Nodetool compactionstats fails with NullPointerException (CASSANDRA-13021)
 + * Thread local pools never cleaned up (CASSANDRA-13033)
 + * Set RPC_READY to false when draining or if a node is marked as shutdown (CASSANDRA-12781)
 + * CQL often queries static columns unnecessarily (CASSANDRA-12768)
 + * Make sure sstables only get committed when it's safe to discard commit log records (CASSANDRA-12956)
 + * Reject default_time_to_live option when creating or altering MVs (CASSANDRA-12868)
 + * Nodetool should use a more sane max heap size (CASSANDRA-12739)
 + * LocalToken ensures token values are cloned on heap (CASSANDRA-12651)
 + * AnticompactionRequestSerializer serializedSize is incorrect (CASSANDRA-12934)
 + * Prevent reloading of logback.xml from UDF sandbox (CASSANDRA-12535)
 + * Reenable HeapPool (CASSANDRA-12900)
-  * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 +Merged from 2.2:
++ * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
++ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
   * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
   * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
   * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)


[10/30] cassandra git commit: Fix race causing infinite loop if Thrift server is stopped before it starts listening

Posted by st...@apache.org.
Fix race causing infinite loop if Thrift server is stopped before it starts listening

patch by Stefania Alborghetti; reviewed by Joel Knighton for CASSANDRA-12856


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

Branch: refs/heads/cassandra-2.2
Commit: 47f7d9c6aab694bf956fa2174fe0dd36702e01a6
Parents: bdca25e
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Tue Jan 3 10:09:52 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:34:15 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 1 +
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/47f7d9c6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a99f79a..23fd12a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.17
+ * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
  * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
  * Fix missed signal when commit log segments are recycled (CASSANDRA-13037)
  * Fix RecoveryManagerTruncateTest (CASSANDRA-12802)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/47f7d9c6/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index 7e67110..905a981 100644
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@ -75,13 +75,15 @@ public class CustomTThreadPoolServer extends TServer
     private final TThreadPoolServer.Args args;
 
     //Track and Limit the number of connected clients
-    private final AtomicInteger activeClients = new AtomicInteger(0);
+    private final AtomicInteger activeClients;
 
 
     public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService) {
         super(args);
         this.executorService = executorService;
+        this.stopped = false;
         this.args = args;
+        this.activeClients = new AtomicInteger(0);
     }
 
     public void serve()
@@ -96,7 +98,6 @@ public class CustomTThreadPoolServer extends TServer
             return;
         }
 
-        stopped = false;
         while (!stopped)
         {
             // block until we are under max clients


[05/30] cassandra git commit: Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

Posted by st...@apache.org.
Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

patch by Jon Haddad; reviewed by Nate McCall for CASSANDRA-12979


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

Branch: refs/heads/trunk
Commit: d8eec1cd6d095961b3754b0de41ffc9061cba370
Parents: 65bd455
Author: Jon Haddad <jo...@jonhaddad.com>
Authored: Wed Jan 4 11:52:54 2017 -0800
Committer: Nate McCall <zz...@gmail.com>
Committed: Tue Jan 10 15:24:36 2017 +1300

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 NEWS.txt                                        |  3 ++
 .../cassandra/db/compaction/CompactionTask.java | 40 +++++++++++++++-----
 3 files changed, 36 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d8eec1cd/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 11b0482..a533e6f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -18,6 +18,7 @@
  * AnticompactionRequestSerializer serializedSize is incorrect (CASSANDRA-12934)
  * Prevent reloading of logback.xml from UDF sandbox (CASSANDRA-12535)
  * Reenable HeapPool (CASSANDRA-12900)
+ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 Merged from 2.2:
  * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
  * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
@@ -34,6 +35,7 @@ Merged from 2.1:
  * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
 
 
+
 3.0.10
  * Disallow offheap_buffers memtable allocation (CASSANDRA-11039)
  * Fix CommitLogSegmentManagerTest (CASSANDRA-12283)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d8eec1cd/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index b4e6551..3b8a333 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -26,6 +26,9 @@ Upgrading
    - Only Java and JavaScript are now supported UDF languages.
      The sandbox in 3.0 already prevented the use of script languages except Java
      and JavaScript.
+   - Compaction now correctly drops sstables out of CompactionTask when there
+     isn't enough disk space to perform the full compaction.  This should reduce
+     pending compaction tasks on systems with little remaining disk space.
 
 3.0.10
 =====

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d8eec1cd/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index f0a1f47..1636ab9 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -90,12 +90,14 @@ public class CompactionTask extends AbstractCompactionTask
         return transaction.originals().size();
     }
 
-    public boolean reduceScopeForLimitedSpace()
+    public boolean reduceScopeForLimitedSpace(long expectedSize)
     {
         if (partialCompactionsAcceptable() && transaction.originals().size() > 1)
         {
             // Try again w/o the largest one.
-            logger.warn("insufficient space to compact all requested files {}", StringUtils.join(transaction.originals(), ", "));
+            logger.warn("insufficient space to compact all requested files. {}MB required, {}",
+                        (float) expectedSize / 1024 / 1024,
+                        StringUtils.join(transaction.originals(), ", "));
             // Note that we have removed files that are still marked as compacting.
             // This suboptimal but ok since the caller will unmark all the sstables at the end.
             SSTableReader removedSSTable = cfs.getMaxSizeFile(transaction.originals());
@@ -128,9 +130,8 @@ public class CompactionTask extends AbstractCompactionTask
 
         // note that we need to do a rough estimate early if we can fit the compaction on disk - this is pessimistic, but
         // since we might remove sstables from the compaction in checkAvailableDiskSpace it needs to be done here
-        long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
-        long earlySSTableEstimate = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
-        checkAvailableDiskSpace(earlySSTableEstimate, expectedWriteSize);
+
+        checkAvailableDiskSpace();
 
         // sanity check: all sstables must belong to the same cfs
         assert !Iterables.any(transaction.originals(), new Predicate<SSTableReader>()
@@ -283,7 +284,12 @@ public class CompactionTask extends AbstractCompactionTask
         return minRepairedAt;
     }
 
-    protected void checkAvailableDiskSpace(long estimatedSSTables, long expectedWriteSize)
+    /*
+    Checks if we have enough disk space to execute the compaction.  Drops the largest sstable out of the Task until
+    there's enough space (in theory) to handle the compaction.  Does not take into account space that will be taken by
+    other compactions.
+     */
+    protected void checkAvailableDiskSpace()
     {
         if(!cfs.isCompactionDiskSpaceCheckEnabled() && compactionType == OperationType.COMPACTION)
         {
@@ -291,10 +297,26 @@ public class CompactionTask extends AbstractCompactionTask
             return;
         }
 
-        while (!getDirectories().hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+        CompactionStrategyManager strategy = cfs.getCompactionStrategyManager();
+
+        while(true)
         {
-            if (!reduceScopeForLimitedSpace())
-                throw new RuntimeException(String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize));
+            long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
+            long estimatedSSTables = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
+
+            if(cfs.getDirectories().hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+                break;
+
+            if (!reduceScopeForLimitedSpace(expectedWriteSize))
+            {
+                // we end up here if we can't take any more sstables out of the compaction.
+                // usually means we've run out of disk space
+                String msg = String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize);
+                logger.warn(msg);
+                throw new RuntimeException(msg);
+            }
+            logger.warn("Not enough space for compaction, {}MB estimated.  Reducing scope.",
+                            (float) expectedWriteSize / 1024 / 1024);
         }
     }
 


[07/30] cassandra git commit: Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

Posted by st...@apache.org.
Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

patch by Jon Haddad; reviewed by Nate McCall for CASSANDRA-12979


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

Branch: refs/heads/cassandra-3.X
Commit: d8eec1cd6d095961b3754b0de41ffc9061cba370
Parents: 65bd455
Author: Jon Haddad <jo...@jonhaddad.com>
Authored: Wed Jan 4 11:52:54 2017 -0800
Committer: Nate McCall <zz...@gmail.com>
Committed: Tue Jan 10 15:24:36 2017 +1300

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 NEWS.txt                                        |  3 ++
 .../cassandra/db/compaction/CompactionTask.java | 40 +++++++++++++++-----
 3 files changed, 36 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d8eec1cd/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 11b0482..a533e6f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -18,6 +18,7 @@
  * AnticompactionRequestSerializer serializedSize is incorrect (CASSANDRA-12934)
  * Prevent reloading of logback.xml from UDF sandbox (CASSANDRA-12535)
  * Reenable HeapPool (CASSANDRA-12900)
+ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 Merged from 2.2:
  * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
  * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
@@ -34,6 +35,7 @@ Merged from 2.1:
  * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
 
 
+
 3.0.10
  * Disallow offheap_buffers memtable allocation (CASSANDRA-11039)
  * Fix CommitLogSegmentManagerTest (CASSANDRA-12283)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d8eec1cd/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index b4e6551..3b8a333 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -26,6 +26,9 @@ Upgrading
    - Only Java and JavaScript are now supported UDF languages.
      The sandbox in 3.0 already prevented the use of script languages except Java
      and JavaScript.
+   - Compaction now correctly drops sstables out of CompactionTask when there
+     isn't enough disk space to perform the full compaction.  This should reduce
+     pending compaction tasks on systems with little remaining disk space.
 
 3.0.10
 =====

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d8eec1cd/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index f0a1f47..1636ab9 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -90,12 +90,14 @@ public class CompactionTask extends AbstractCompactionTask
         return transaction.originals().size();
     }
 
-    public boolean reduceScopeForLimitedSpace()
+    public boolean reduceScopeForLimitedSpace(long expectedSize)
     {
         if (partialCompactionsAcceptable() && transaction.originals().size() > 1)
         {
             // Try again w/o the largest one.
-            logger.warn("insufficient space to compact all requested files {}", StringUtils.join(transaction.originals(), ", "));
+            logger.warn("insufficient space to compact all requested files. {}MB required, {}",
+                        (float) expectedSize / 1024 / 1024,
+                        StringUtils.join(transaction.originals(), ", "));
             // Note that we have removed files that are still marked as compacting.
             // This suboptimal but ok since the caller will unmark all the sstables at the end.
             SSTableReader removedSSTable = cfs.getMaxSizeFile(transaction.originals());
@@ -128,9 +130,8 @@ public class CompactionTask extends AbstractCompactionTask
 
         // note that we need to do a rough estimate early if we can fit the compaction on disk - this is pessimistic, but
         // since we might remove sstables from the compaction in checkAvailableDiskSpace it needs to be done here
-        long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
-        long earlySSTableEstimate = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
-        checkAvailableDiskSpace(earlySSTableEstimate, expectedWriteSize);
+
+        checkAvailableDiskSpace();
 
         // sanity check: all sstables must belong to the same cfs
         assert !Iterables.any(transaction.originals(), new Predicate<SSTableReader>()
@@ -283,7 +284,12 @@ public class CompactionTask extends AbstractCompactionTask
         return minRepairedAt;
     }
 
-    protected void checkAvailableDiskSpace(long estimatedSSTables, long expectedWriteSize)
+    /*
+    Checks if we have enough disk space to execute the compaction.  Drops the largest sstable out of the Task until
+    there's enough space (in theory) to handle the compaction.  Does not take into account space that will be taken by
+    other compactions.
+     */
+    protected void checkAvailableDiskSpace()
     {
         if(!cfs.isCompactionDiskSpaceCheckEnabled() && compactionType == OperationType.COMPACTION)
         {
@@ -291,10 +297,26 @@ public class CompactionTask extends AbstractCompactionTask
             return;
         }
 
-        while (!getDirectories().hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+        CompactionStrategyManager strategy = cfs.getCompactionStrategyManager();
+
+        while(true)
         {
-            if (!reduceScopeForLimitedSpace())
-                throw new RuntimeException(String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize));
+            long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
+            long estimatedSSTables = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
+
+            if(cfs.getDirectories().hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+                break;
+
+            if (!reduceScopeForLimitedSpace(expectedWriteSize))
+            {
+                // we end up here if we can't take any more sstables out of the compaction.
+                // usually means we've run out of disk space
+                String msg = String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize);
+                logger.warn(msg);
+                throw new RuntimeException(msg);
+            }
+            logger.warn("Not enough space for compaction, {}MB estimated.  Reducing scope.",
+                            (float) expectedWriteSize / 1024 / 1024);
         }
     }
 


[26/30] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by st...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


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

Branch: refs/heads/cassandra-3.11
Commit: b52c820454619e2c007c8d312482e860132aae5c
Parents: e9251e1 6ff56c3
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jan 11 07:52:03 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:52:03 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 3 ++-
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b52c8204/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 99f6613,5f37cd7..b8250ab
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,117 -1,6 +1,116 @@@
 -3.0.11
 +3.10
 + * Fixed query monitoring for range queries (CASSANDRA-13050)
 + * Remove outboundBindAny configuration property (CASSANDRA-12673)
 + * Use correct bounds for all-data range when filtering (CASSANDRA-12666)
 + * Remove timing window in test case (CASSANDRA-12875)
 + * Resolve unit testing without JCE security libraries installed (CASSANDRA-12945)
 + * Fix inconsistencies in cassandra-stress load balancing policy (CASSANDRA-12919)
 + * Fix validation of non-frozen UDT cells (CASSANDRA-12916)
 + * Don't shut down socket input/output on StreamSession (CASSANDRA-12903)
 + * Fix Murmur3PartitionerTest (CASSANDRA-12858)
 + * Move cqlsh syntax rules into separate module and allow easier customization (CASSANDRA-12897)
 + * Fix CommitLogSegmentManagerTest (CASSANDRA-12283)
 + * Fix cassandra-stress truncate option (CASSANDRA-12695)
 + * Fix crossNode value when receiving messages (CASSANDRA-12791)
 + * Don't load MX4J beans twice (CASSANDRA-12869)
 + * Extend native protocol request flags, add versions to SUPPORTED, and introduce ProtocolVersion enum (CASSANDRA-12838)
 + * Set JOINING mode when running pre-join tasks (CASSANDRA-12836)
 + * remove net.mintern.primitive library due to license issue (CASSANDRA-12845)
 + * Properly format IPv6 addresses when logging JMX service URL (CASSANDRA-12454)
 + * Optimize the vnode allocation for single replica per DC (CASSANDRA-12777)
 + * Use non-token restrictions for bounds when token restrictions are overridden (CASSANDRA-12419)
 + * Fix CQLSH auto completion for PER PARTITION LIMIT (CASSANDRA-12803)
 + * Use different build directories for Eclipse and Ant (CASSANDRA-12466)
 + * Avoid potential AttributeError in cqlsh due to no table metadata (CASSANDRA-12815)
 + * Fix RandomReplicationAwareTokenAllocatorTest.testExistingCluster (CASSANDRA-12812)
 + * Upgrade commons-codec to 1.9 (CASSANDRA-12790)
 + * Make the fanout size for LeveledCompactionStrategy to be configurable (CASSANDRA-11550)
 + * Add duration data type (CASSANDRA-11873)
 + * Fix timeout in ReplicationAwareTokenAllocatorTest (CASSANDRA-12784)
 + * Improve sum aggregate functions (CASSANDRA-12417)
 + * Make cassandra.yaml docs for batch_size_*_threshold_in_kb reflect changes in CASSANDRA-10876 (CASSANDRA-12761)
 + * cqlsh fails to format collections when using aliases (CASSANDRA-11534)
 + * Check for hash conflicts in prepared statements (CASSANDRA-12733)
 + * Exit query parsing upon first error (CASSANDRA-12598)
 + * Fix cassandra-stress to use single seed in UUID generation (CASSANDRA-12729)
 + * CQLSSTableWriter does not allow Update statement (CASSANDRA-12450)
 + * Config class uses boxed types but DD exposes primitive types (CASSANDRA-12199)
 + * Add pre- and post-shutdown hooks to Storage Service (CASSANDRA-12461)
 + * Add hint delivery metrics (CASSANDRA-12693)
 + * Remove IndexInfo cache from FileIndexInfoRetriever (CASSANDRA-12731)
 + * ColumnIndex does not reuse buffer (CASSANDRA-12502)
 + * cdc column addition still breaks schema migration tasks (CASSANDRA-12697)
 + * Upgrade metrics-reporter dependencies (CASSANDRA-12089)
 + * Tune compaction thread count via nodetool (CASSANDRA-12248)
 + * Add +=/-= shortcut syntax for update queries (CASSANDRA-12232)
 + * Include repair session IDs in repair start message (CASSANDRA-12532)
 + * Add a blocking task to Index, run before joining the ring (CASSANDRA-12039)
 + * Fix NPE when using CQLSSTableWriter (CASSANDRA-12667)
 + * Support optional backpressure strategies at the coordinator (CASSANDRA-9318)
 + * Make randompartitioner work with new vnode allocation (CASSANDRA-12647)
 + * Fix cassandra-stress graphing (CASSANDRA-12237)
 + * Allow filtering on partition key columns for queries without secondary indexes (CASSANDRA-11031)
 + * Fix Cassandra Stress reporting thread model and precision (CASSANDRA-12585)
 + * Add JMH benchmarks.jar (CASSANDRA-12586)
 + * Cleanup uses of AlterTableStatementColumn (CASSANDRA-12567)
 + * Add keep-alive to streaming (CASSANDRA-11841)
 + * Tracing payload is passed through newSession(..) (CASSANDRA-11706)
 + * avoid deleting non existing sstable files and improve related log messages (CASSANDRA-12261)
 + * json/yaml output format for nodetool compactionhistory (CASSANDRA-12486)
 + * Retry all internode messages once after a connection is
 +   closed and reopened (CASSANDRA-12192)
 + * Add support to rebuild from targeted replica (CASSANDRA-9875)
 + * Add sequence distribution type to cassandra stress (CASSANDRA-12490)
 + * "SELECT * FROM foo LIMIT ;" does not error out (CASSANDRA-12154)
 + * Define executeLocally() at the ReadQuery Level (CASSANDRA-12474)
 + * Extend read/write failure messages with a map of replica addresses
 +   to error codes in the v5 native protocol (CASSANDRA-12311)
 + * Fix rebuild of SASI indexes with existing index files (CASSANDRA-12374)
 + * Let DatabaseDescriptor not implicitly startup services (CASSANDRA-9054, 12550)
 + * Fix clustering indexes in presence of static columns in SASI (CASSANDRA-12378)
 + * Fix queries on columns with reversed type on SASI indexes (CASSANDRA-12223)
 + * Added slow query log (CASSANDRA-12403)
 + * Count full coordinated request against timeout (CASSANDRA-12256)
 + * Allow TTL with null value on insert and update (CASSANDRA-12216)
 + * Make decommission operation resumable (CASSANDRA-12008)
 + * Add support to one-way targeted repair (CASSANDRA-9876)
 + * Remove clientutil jar (CASSANDRA-11635)
 + * Fix compaction throughput throttle (CASSANDRA-12366, CASSANDRA-12717)
 + * Delay releasing Memtable memory on flush until PostFlush has finished running (CASSANDRA-12358)
 + * Cassandra stress should dump all setting on startup (CASSANDRA-11914)
 + * Make it possible to compact a given token range (CASSANDRA-10643)
 + * Allow updating DynamicEndpointSnitch properties via JMX (CASSANDRA-12179)
 + * Collect metrics on queries by consistency level (CASSANDRA-7384)
 + * Add support for GROUP BY to SELECT statement (CASSANDRA-10707)
 + * Deprecate memtable_cleanup_threshold and update default for memtable_flush_writers (CASSANDRA-12228)
 + * Upgrade to OHC 0.4.4 (CASSANDRA-12133)
 + * Add version command to cassandra-stress (CASSANDRA-12258)
 + * Create compaction-stress tool (CASSANDRA-11844)
 + * Garbage-collecting compaction operation and schema option (CASSANDRA-7019)
 + * Add beta protocol flag for v5 native protocol (CASSANDRA-12142)
 + * Support filtering on non-PRIMARY KEY columns in the CREATE
 +   MATERIALIZED VIEW statement's WHERE clause (CASSANDRA-10368)
 + * Unify STDOUT and SYSTEMLOG logback format (CASSANDRA-12004)
 + * COPY FROM should raise error for non-existing input files (CASSANDRA-12174)
 + * Faster write path (CASSANDRA-12269)
 + * Option to leave omitted columns in INSERT JSON unset (CASSANDRA-11424)
 + * Support json/yaml output in nodetool tpstats (CASSANDRA-12035)
 + * Expose metrics for successful/failed authentication attempts (CASSANDRA-10635)
 + * Prepend snapshot name with "truncated" or "dropped" when a snapshot
 +   is taken before truncating or dropping a table (CASSANDRA-12178)
 + * Optimize RestrictionSet (CASSANDRA-12153)
 + * cqlsh does not automatically downgrade CQL version (CASSANDRA-12150)
 + * Omit (de)serialization of state variable in UDAs (CASSANDRA-9613)
 + * Create a system table to expose prepared statements (CASSANDRA-8831)
 + * Reuse DataOutputBuffer from ColumnIndex (CASSANDRA-11970)
 + * Remove DatabaseDescriptor dependency from SegmentedFile (CASSANDRA-11580)
 + * Add supplied username to authentication error messages (CASSANDRA-12076)
 + * Remove pre-startup check for open JMX port (CASSANDRA-12074)
 + * Remove compaction Severity from DynamicEndpointSnitch (CASSANDRA-11738)
 + * Restore resumable hints delivery (CASSANDRA-11960)
 + * Properly report LWT contention (CASSANDRA-12626)
 +Merged from 3.0:
-  * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
   * Replace empty strings with null values if they cannot be converted (CASSANDRA-12794)
 - * Fixed flacky SSTableRewriterTest: check file counts before calling validateCFS (CASSANDRA-12348)
   * Fix deserialization of 2.x DeletedCells (CASSANDRA-12620)
   * Add parent repair session id to anticompaction log message (CASSANDRA-12186)
   * Improve contention handling on failure to acquire MV lock for streaming and hints (CASSANDRA-12905)
@@@ -177,16 -73,6 +176,18 @@@
   * Correct log message for statistics of offheap memtable flush (CASSANDRA-12776)
   * Explicitly set locale for string validation (CASSANDRA-12541,CASSANDRA-12542,CASSANDRA-12543,CASSANDRA-12545)
  Merged from 2.2:
++ * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
++ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 + * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
 + * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
 + * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)
 + * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
 + * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
 + * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
 + * Use saved tokens when setting local tokens on StorageService.joinRing (CASSANDRA-12935)
 + * cqlsh: fix DESC TYPES errors (CASSANDRA-12914)
 + * Fix leak on skipped SSTables in sstableupgrade (CASSANDRA-12899)
 + * Avoid blocking gossip during pending range calculation (CASSANDRA-12281)
   * Fix purgeability of tombstones with max timestamp (CASSANDRA-12792)
   * Fail repair if participant dies during sync or anticompaction (CASSANDRA-12901)
   * cqlsh COPY: unprotected pk values before converting them if not using prepared statements (CASSANDRA-12863)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b52c8204/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index d1cdaed,efa9330..31aa689
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@@ -75,14 -75,15 +75,16 @@@ public class CustomTThreadPoolServer ex
      private final TThreadPoolServer.Args args;
  
      //Track and Limit the number of connected clients
-     private final AtomicInteger activeClients = new AtomicInteger(0);
+     private final AtomicInteger activeClients;
  
  
 -    public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService) {
 +    public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService)
 +    {
          super(args);
          this.executorService = executorService;
+         this.stopped = false;
          this.args = args;
+         this.activeClients = new AtomicInteger(0);
      }
  
      @SuppressWarnings("resource")


[03/30] cassandra git commit: Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

Posted by st...@apache.org.
Fixed checkAvailableDiskSpace to properly recalculate expected disk usage of compaction task before reducing scope

patch by Jon Haddad; reviewed by Nate McCall for CASSANDRA-12979


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

Branch: refs/heads/trunk
Commit: dbe039f628220e5141836ecb9aab05d4da60589a
Parents: 6f360b6
Author: Jon Haddad <jo...@jonhaddad.com>
Authored: Wed Jan 4 11:52:54 2017 -0800
Committer: Nate McCall <zz...@gmail.com>
Committed: Tue Jan 10 15:15:47 2017 +1300

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  6 +++
 .../cassandra/db/compaction/CompactionTask.java | 40 +++++++++++++++-----
 3 files changed, 38 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbe039f6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b41313d..4b301ee 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -24,6 +24,7 @@
  * Make Collections deserialization more robust (CASSANDRA-12618)
  * Better handle invalid system roles table (CASSANDRA-12700)
  * Split consistent range movement flag correction (CASSANDRA-12786)
+ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 Merged from 2.1:
  * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
  * Don't skip sstables based on maxLocalDeletionTime (CASSANDRA-12765)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbe039f6/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 37949a1..06e9eeb 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -16,6 +16,12 @@ using the provided 'sstableupgrade' tool.
 2.2.9
 =====
 
+Upgrading
+---------
+   - Compaction now correctly drops sstables out of CompactionTask when there
+     isn't enough disk space to perform the full compaction.  This should reduce
+     pending compaction tasks on systems with little remaining disk space.
+
 Deprecation
 -----------
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dbe039f6/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 20d3dc0..7489b3d 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -75,12 +75,14 @@ public class CompactionTask extends AbstractCompactionTask
         return transaction.originals().size();
     }
 
-    public boolean reduceScopeForLimitedSpace()
+    public boolean reduceScopeForLimitedSpace(long expectedSize)
     {
         if (partialCompactionsAcceptable() && transaction.originals().size() > 1)
         {
             // Try again w/o the largest one.
-            logger.warn("insufficient space to compact all requested files {}", StringUtils.join(transaction.originals(), ", "));
+            logger.warn("insufficient space to compact all requested files. {}MB required, {}",
+                        (float) expectedSize / 1024 / 1024,
+                        StringUtils.join(transaction.originals(), ", "));
             // Note that we have removed files that are still marked as compacting.
             // This suboptimal but ok since the caller will unmark all the sstables at the end.
             SSTableReader removedSSTable = cfs.getMaxSizeFile(transaction.originals());
@@ -113,9 +115,8 @@ public class CompactionTask extends AbstractCompactionTask
 
         // note that we need to do a rough estimate early if we can fit the compaction on disk - this is pessimistic, but
         // since we might remove sstables from the compaction in checkAvailableDiskSpace it needs to be done here
-        long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
-        long earlySSTableEstimate = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
-        checkAvailableDiskSpace(earlySSTableEstimate, expectedWriteSize);
+
+        checkAvailableDiskSpace();
 
         // sanity check: all sstables must belong to the same cfs
         assert !Iterables.any(transaction.originals(), new Predicate<SSTableReader>()
@@ -271,12 +272,33 @@ public class CompactionTask extends AbstractCompactionTask
         return minRepairedAt;
     }
 
-    protected void checkAvailableDiskSpace(long estimatedSSTables, long expectedWriteSize)
+    /*
+    Checks if we have enough disk space to execute the compaction.  Drops the largest sstable out of the Task until
+    there's enough space (in theory) to handle the compaction.  Does not take into account space that will be taken by
+    other compactions.
+     */
+    protected void checkAvailableDiskSpace()
     {
-        while (!cfs.directories.hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+        AbstractCompactionStrategy strategy = cfs.getCompactionStrategy();
+
+        while(true)
         {
-            if (!reduceScopeForLimitedSpace())
-                throw new RuntimeException(String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize));
+            long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
+            long estimatedSSTables = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
+
+            if(cfs.directories.hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+                break;
+
+            if (!reduceScopeForLimitedSpace(expectedWriteSize))
+            {
+                // we end up here if we can't take any more sstables out of the compaction.
+                // usually means we've run out of disk space
+                String msg = String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize);
+                logger.warn(msg);
+                throw new RuntimeException(msg);
+            }
+            logger.warn("Not enough space for compaction, {}MB estimated.  Reducing scope.",
+                            (float) expectedWriteSize / 1024 / 1024);
         }
     }
 


[16/30] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by st...@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/2acc15ba
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2acc15ba
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2acc15ba

Branch: refs/heads/cassandra-3.11
Commit: 2acc15ba38695495acdb678dcb005d7f611bae2b
Parents: dbe039f 47f7d9c
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jan 11 07:37:22 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:37:22 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 1 +
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2acc15ba/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 4b301ee,23fd12a..a7e89ed
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,32 -1,8 +1,33 @@@
 -2.1.17
 +2.2.9
 + * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
 + * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
 + * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)
 + * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
 + * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
 + * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
 + * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)
 + * Use saved tokens when setting local tokens on StorageService.joinRing (CASSANDRA-12935)
 + * cqlsh: fix DESC TYPES errors (CASSANDRA-12914)
 + * Fix leak on skipped SSTables in sstableupgrade (CASSANDRA-12899)
 + * Avoid blocking gossip during pending range calculation (CASSANDRA-12281)
 + * Fix purgeability of tombstones with max timestamp (CASSANDRA-12792)
 + * Fail repair if participant dies during sync or anticompaction (CASSANDRA-12901)
 + * cqlsh COPY: unprotected pk values before converting them if not using prepared statements (CASSANDRA-12863)
 + * Fix Util.spinAssertEquals (CASSANDRA-12283)
 + * Fix potential NPE for compactionstats (CASSANDRA-12462)
 + * Prepare legacy authenticate statement if credentials table initialised after node startup (CASSANDRA-12813)
 + * Change cassandra.wait_for_tracing_events_timeout_secs default to 0 (CASSANDRA-12754)
 + * Clean up permissions when a UDA is dropped (CASSANDRA-12720)
 + * Limit colUpdateTimeDelta histogram updates to reasonable deltas (CASSANDRA-11117)
 + * Fix leak errors and execution rejected exceptions when draining (CASSANDRA-12457)
 + * Fix merkle tree depth calculation (CASSANDRA-12580)
 + * Make Collections deserialization more robust (CASSANDRA-12618)
 + * Better handle invalid system roles table (CASSANDRA-12700)
 + * Split consistent range movement flag correction (CASSANDRA-12786)
 + * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 +Merged from 2.1:
+  * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
   * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
 - * Fix missed signal when commit log segments are recycled (CASSANDRA-13037)
 - * Fix RecoveryManagerTruncateTest (CASSANDRA-12802)
   * Don't skip sstables based on maxLocalDeletionTime (CASSANDRA-12765)
  
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2acc15ba/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index c5f34ae,905a981..efa9330
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@@ -81,10 -81,11 +81,12 @@@ public class CustomTThreadPoolServer ex
      public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService) {
          super(args);
          this.executorService = executorService;
+         this.stopped = false;
          this.args = args;
+         this.activeClients = new AtomicInteger(0);
      }
  
 +    @SuppressWarnings("resource")
      public void serve()
      {
          try


[22/30] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by st...@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/6ff56c30
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6ff56c30
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6ff56c30

Branch: refs/heads/cassandra-3.X
Commit: 6ff56c30e0913b49291a5a61a49eacac8ee3eb28
Parents: d8eec1c 2acc15b
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jan 11 07:45:27 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:47:06 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 3 ++-
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6ff56c30/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index a533e6f,a7e89ed..5f37cd7
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,25 -1,4 +1,26 @@@
 -2.2.9
 +3.0.11
 + * Replace empty strings with null values if they cannot be converted (CASSANDRA-12794)
 + * Fixed flacky SSTableRewriterTest: check file counts before calling validateCFS (CASSANDRA-12348)
 + * Fix deserialization of 2.x DeletedCells (CASSANDRA-12620)
 + * Add parent repair session id to anticompaction log message (CASSANDRA-12186)
 + * Improve contention handling on failure to acquire MV lock for streaming and hints (CASSANDRA-12905)
 + * Fix DELETE and UPDATE queries with empty IN restrictions (CASSANDRA-12829)
 + * Mark MVs as built after successful bootstrap (CASSANDRA-12984)
 + * Estimated TS drop-time histogram updated with Cell.NO_DELETION_TIME (CASSANDRA-13040)
 + * Nodetool compactionstats fails with NullPointerException (CASSANDRA-13021)
 + * Thread local pools never cleaned up (CASSANDRA-13033)
 + * Set RPC_READY to false when draining or if a node is marked as shutdown (CASSANDRA-12781)
 + * CQL often queries static columns unnecessarily (CASSANDRA-12768)
 + * Make sure sstables only get committed when it's safe to discard commit log records (CASSANDRA-12956)
 + * Reject default_time_to_live option when creating or altering MVs (CASSANDRA-12868)
 + * Nodetool should use a more sane max heap size (CASSANDRA-12739)
 + * LocalToken ensures token values are cloned on heap (CASSANDRA-12651)
 + * AnticompactionRequestSerializer serializedSize is incorrect (CASSANDRA-12934)
 + * Prevent reloading of logback.xml from UDF sandbox (CASSANDRA-12535)
 + * Reenable HeapPool (CASSANDRA-12900)
-  * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 +Merged from 2.2:
++ * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
++ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
   * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
   * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
   * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)


[14/30] cassandra git commit: Fix race causing infinite loop if Thrift server is stopped before it starts listening

Posted by st...@apache.org.
Fix race causing infinite loop if Thrift server is stopped before it starts listening

patch by Stefania Alborghetti; reviewed by Joel Knighton for CASSANDRA-12856


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

Branch: refs/heads/cassandra-3.11
Commit: 47f7d9c6aab694bf956fa2174fe0dd36702e01a6
Parents: bdca25e
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Tue Jan 3 10:09:52 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:34:15 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                     | 1 +
 .../org/apache/cassandra/thrift/CustomTThreadPoolServer.java    | 5 +++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/47f7d9c6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a99f79a..23fd12a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.17
+ * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
  * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
  * Fix missed signal when commit log segments are recycled (CASSANDRA-13037)
  * Fix RecoveryManagerTruncateTest (CASSANDRA-12802)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/47f7d9c6/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index 7e67110..905a981 100644
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@ -75,13 +75,15 @@ public class CustomTThreadPoolServer extends TServer
     private final TThreadPoolServer.Args args;
 
     //Track and Limit the number of connected clients
-    private final AtomicInteger activeClients = new AtomicInteger(0);
+    private final AtomicInteger activeClients;
 
 
     public CustomTThreadPoolServer(TThreadPoolServer.Args args, ExecutorService executorService) {
         super(args);
         this.executorService = executorService;
+        this.stopped = false;
         this.args = args;
+        this.activeClients = new AtomicInteger(0);
     }
 
     public void serve()
@@ -96,7 +98,6 @@ public class CustomTThreadPoolServer extends TServer
             return;
         }
 
-        stopped = false;
         while (!stopped)
         {
             // block until we are under max clients


[09/30] cassandra git commit: Merge branch 'cassandra-3.11' into cassandra-3.X

Posted by st...@apache.org.
Merge branch 'cassandra-3.11' into cassandra-3.X


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

Branch: refs/heads/trunk
Commit: d1d75ad8ce1a8e76122cee39cf8201362e63cd75
Parents: dfd8dcf e9251e1
Author: Nate McCall <zz...@gmail.com>
Authored: Tue Jan 10 16:27:47 2017 +1300
Committer: Nate McCall <zz...@gmail.com>
Committed: Tue Jan 10 16:32:39 2017 +1300

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +-
 NEWS.txt                                        |  3 ++
 .../cassandra/db/compaction/CompactionTask.java | 41 +++++++++++++++-----
 3 files changed, 36 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d1d75ad8/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index b47ba37,99f6613..ba4172b
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,26 -1,3 +1,26 @@@
 +3.12
 + * Use new token allocation for non bootstrap case as well (CASSANDRA-13080)
 + * Avoid byte-array copy when key cache is disabled (CASSANDRA-13084)
 + * More fixes to the TokenAllocator (CASSANDRA-12990)
 + * Require forceful decommission if number of nodes is less than replication factor (CASSANDRA-12510)
 + * Allow IN restrictions on column families with collections (CASSANDRA-12654)
 + * Move to FastThreadLocalThread and FastThreadLocal (CASSANDRA-13034)
 + * nodetool stopdaemon errors out (CASSANDRA-13030)
 + * Log message size in trace message in OutboundTcpConnection (CASSANDRA-13028)
 + * Add timeUnit Days for cassandra-stress (CASSANDRA-13029)
 + * Add mutation size and batch metrics (CASSANDRA-12649)
 + * Add method to get size of endpoints to TokenMetadata (CASSANDRA-12999)
 + * Fix primary index calculation for SASI (CASSANDRA-12910)
 + * Expose time spent waiting in thread pool queue (CASSANDRA-8398)
 + * Conditionally update index built status to avoid unnecessary flushes (CASSANDRA-12969)
 + * NoReplicationTokenAllocator should work with zero replication factor (CASSANDRA-12983)
 + * cqlsh auto completion: refactor definition of compaction strategy options (CASSANDRA-12946)
 + * Add support for arithmetic operators (CASSANDRA-11935)
 + * Tables in system_distributed should not use gcgs of 0 (CASSANDRA-12954)
 +Merged from 3.0:
 + * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
- 
++ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 +
  3.10
   * Fixed query monitoring for range queries (CASSANDRA-13050)
   * Remove outboundBindAny configuration property (CASSANDRA-12673)


[30/30] cassandra git commit: Merge branch 'cassandra-3.X' into trunk

Posted by st...@apache.org.
Merge branch 'cassandra-3.X' into trunk


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

Branch: refs/heads/trunk
Commit: 87b0b36c755d6d12522c8860e8d442edafdc80a1
Parents: b3ffdf8 699eff0
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Wed Jan 11 07:59:48 2017 +0100
Committer: Stefania Alborghetti <st...@datastax.com>
Committed: Wed Jan 11 07:59:48 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/87b0b36c/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index d0c4960,917eb5d..3ed8e15
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,15 -1,3 +1,14 @@@
 +4.0
 + * Use Caffeine (W-TinyLFU) for on-heap caches (CASSANDRA-10855)
 + * Thrift removal (CASSANDRA-11115)
 + * Remove pre-3.0 compatibility code for 4.0 (CASSANDRA-12716)
 + * Add column definition kind to dropped columns in schema (CASSANDRA-12705)
 + * Add (automate) Nodetool Documentation (CASSANDRA-12672)
 + * Update bundled cqlsh python driver to 3.7.0 (CASSANDRA-12736)
 + * Reject invalid replication settings when creating or altering a keyspace (CASSANDRA-12681)
 + * Clean up the SSTableReader#getScanner API wrt removal of RateLimiter (CASSANDRA-12422)
-  * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
 +
 +
  3.12
   * Use new token allocation for non bootstrap case as well (CASSANDRA-13080)
   * Avoid byte-array copy when key cache is disabled (CASSANDRA-13084)
@@@ -29,9 -17,8 +28,7 @@@
   * cqlsh auto completion: refactor definition of compaction strategy options (CASSANDRA-12946)
   * Add support for arithmetic operators (CASSANDRA-11935)
   * Tables in system_distributed should not use gcgs of 0 (CASSANDRA-12954)
- Merged from 3.0:
-  * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
  
 -
  3.10
   * Fixed query monitoring for range queries (CASSANDRA-13050)
   * Remove outboundBindAny configuration property (CASSANDRA-12673)
@@@ -142,9 -129,10 +139,10 @@@
   * Remove pre-startup check for open JMX port (CASSANDRA-12074)
   * Remove compaction Severity from DynamicEndpointSnitch (CASSANDRA-11738)
   * Restore resumable hints delivery (CASSANDRA-11960)
 - * Properly report LWT contention (CASSANDRA-12626)
 + * Properly record CAS contention (CASSANDRA-12626)
  Merged from 3.0:
   * Replace empty strings with null values if they cannot be converted (CASSANDRA-12794)
+  * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
   * Fix deserialization of 2.x DeletedCells (CASSANDRA-12620)
   * Add parent repair session id to anticompaction log message (CASSANDRA-12186)
   * Improve contention handling on failure to acquire MV lock for streaming and hints (CASSANDRA-12905)