You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by pa...@apache.org on 2018/01/10 20:43:15 UTC

[1/3] cassandra git commit: Acquire read lock before accessing CompactionStrategyManager fields

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.11 62e46f719 -> fe0ee85c7
  refs/heads/trunk ee907a321 -> 7a29d22e6


Acquire read lock before accessing CompactionStrategyManager fields

Patch by Paulo Motta; Reviewed by Marcus Eriksson for CASSANDRA-14139


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

Branch: refs/heads/cassandra-3.11
Commit: fe0ee85c71faada0acb48a65f249575c65bf0972
Parents: 62e46f7
Author: Paulo Motta <pa...@gmail.com>
Authored: Sat Dec 30 02:10:35 2017 +1100
Committer: Paulo Motta <pa...@apache.org>
Committed: Wed Jan 10 18:39:12 2018 -0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../compaction/CompactionStrategyManager.java   | 47 ++++++++++++++------
 2 files changed, 34 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe0ee85c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index d6d8066..b89ad99 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.11.2
+ * Acquire read lock before accessing CompactionStrategyManager fields (CASSANDRA-14139)
  * Split CommitLogStressTest to avoid timeout (CASSANDRA-14143)
  * Avoid invalidating disk boundaries unnecessarily (CASSANDRA-14083)
  * Avoid exposing compaction strategy index externally (CASSANDRA-14082)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe0ee85c/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
index 39d253b..a50f428 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
@@ -105,6 +105,7 @@ public class CompactionStrategyManager implements INotificationConsumer
      **/
     private volatile CompactionParams schemaCompactionParams;
     private boolean shouldDefragment;
+    private boolean supportsEarlyOpen;
     private int fanout;
 
     public CompactionStrategyManager(ColumnFamilyStore cfs)
@@ -216,6 +217,7 @@ public class CompactionStrategyManager implements INotificationConsumer
             repaired.forEach(AbstractCompactionStrategy::startup);
             unrepaired.forEach(AbstractCompactionStrategy::startup);
             shouldDefragment = repaired.get(0).shouldDefragment();
+            supportsEarlyOpen = repaired.get(0).supportsEarlyOpen();
             fanout = (repaired.get(0) instanceof LeveledCompactionStrategy) ? ((LeveledCompactionStrategy) repaired.get(0)).getLevelFanoutSize() : LeveledCompactionStrategy.DEFAULT_LEVEL_FANOUT_SIZE;
         }
         finally
@@ -1037,35 +1039,52 @@ public class CompactionStrategyManager implements INotificationConsumer
 
     public boolean isRepaired(AbstractCompactionStrategy strategy)
     {
-        return repaired.contains(strategy);
+        readLock.lock();
+        try
+        {
+            return repaired.contains(strategy);
+        }
+        finally
+        {
+            readLock.unlock();
+        }
     }
 
     public List<String> getStrategyFolders(AbstractCompactionStrategy strategy)
     {
-        List<Directories.DataDirectory> locations = currentBoundaries.directories;
-        if (partitionSSTablesByTokenRange)
+        readLock.lock();
+        try
         {
-            int unrepairedIndex = unrepaired.indexOf(strategy);
-            if (unrepairedIndex > 0)
+            List<Directories.DataDirectory> locations = currentBoundaries.directories;
+            if (partitionSSTablesByTokenRange)
             {
-                return Collections.singletonList(locations.get(unrepairedIndex).location.getAbsolutePath());
+                int unrepairedIndex = unrepaired.indexOf(strategy);
+                if (unrepairedIndex > 0)
+                {
+                    return Collections.singletonList(locations.get(unrepairedIndex).location.getAbsolutePath());
+                }
+                int repairedIndex = repaired.indexOf(strategy);
+                if (repairedIndex > 0)
+                {
+                    return Collections.singletonList(locations.get(repairedIndex).location.getAbsolutePath());
+                }
             }
-            int repairedIndex = repaired.indexOf(strategy);
-            if (repairedIndex > 0)
+            List<String> folders = new ArrayList<>(locations.size());
+            for (Directories.DataDirectory location : locations)
             {
-                return Collections.singletonList(locations.get(repairedIndex).location.getAbsolutePath());
+                folders.add(location.location.getAbsolutePath());
             }
+            return folders;
         }
-        List<String> folders = new ArrayList<>(locations.size());
-        for (Directories.DataDirectory location : locations)
+        finally
         {
-            folders.add(location.location.getAbsolutePath());
+            readLock.unlock();
         }
-        return folders;
+
     }
 
     public boolean supportsEarlyOpen()
     {
-        return repaired.get(0).supportsEarlyOpen();
+        return supportsEarlyOpen;
     }
 }


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


[2/3] cassandra git commit: Acquire read lock before accessing CompactionStrategyManager fields

Posted by pa...@apache.org.
Acquire read lock before accessing CompactionStrategyManager fields

Patch by Paulo Motta; Reviewed by Marcus Eriksson for CASSANDRA-14139


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

Branch: refs/heads/trunk
Commit: fe0ee85c71faada0acb48a65f249575c65bf0972
Parents: 62e46f7
Author: Paulo Motta <pa...@gmail.com>
Authored: Sat Dec 30 02:10:35 2017 +1100
Committer: Paulo Motta <pa...@apache.org>
Committed: Wed Jan 10 18:39:12 2018 -0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../compaction/CompactionStrategyManager.java   | 47 ++++++++++++++------
 2 files changed, 34 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe0ee85c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index d6d8066..b89ad99 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.11.2
+ * Acquire read lock before accessing CompactionStrategyManager fields (CASSANDRA-14139)
  * Split CommitLogStressTest to avoid timeout (CASSANDRA-14143)
  * Avoid invalidating disk boundaries unnecessarily (CASSANDRA-14083)
  * Avoid exposing compaction strategy index externally (CASSANDRA-14082)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe0ee85c/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
index 39d253b..a50f428 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
@@ -105,6 +105,7 @@ public class CompactionStrategyManager implements INotificationConsumer
      **/
     private volatile CompactionParams schemaCompactionParams;
     private boolean shouldDefragment;
+    private boolean supportsEarlyOpen;
     private int fanout;
 
     public CompactionStrategyManager(ColumnFamilyStore cfs)
@@ -216,6 +217,7 @@ public class CompactionStrategyManager implements INotificationConsumer
             repaired.forEach(AbstractCompactionStrategy::startup);
             unrepaired.forEach(AbstractCompactionStrategy::startup);
             shouldDefragment = repaired.get(0).shouldDefragment();
+            supportsEarlyOpen = repaired.get(0).supportsEarlyOpen();
             fanout = (repaired.get(0) instanceof LeveledCompactionStrategy) ? ((LeveledCompactionStrategy) repaired.get(0)).getLevelFanoutSize() : LeveledCompactionStrategy.DEFAULT_LEVEL_FANOUT_SIZE;
         }
         finally
@@ -1037,35 +1039,52 @@ public class CompactionStrategyManager implements INotificationConsumer
 
     public boolean isRepaired(AbstractCompactionStrategy strategy)
     {
-        return repaired.contains(strategy);
+        readLock.lock();
+        try
+        {
+            return repaired.contains(strategy);
+        }
+        finally
+        {
+            readLock.unlock();
+        }
     }
 
     public List<String> getStrategyFolders(AbstractCompactionStrategy strategy)
     {
-        List<Directories.DataDirectory> locations = currentBoundaries.directories;
-        if (partitionSSTablesByTokenRange)
+        readLock.lock();
+        try
         {
-            int unrepairedIndex = unrepaired.indexOf(strategy);
-            if (unrepairedIndex > 0)
+            List<Directories.DataDirectory> locations = currentBoundaries.directories;
+            if (partitionSSTablesByTokenRange)
             {
-                return Collections.singletonList(locations.get(unrepairedIndex).location.getAbsolutePath());
+                int unrepairedIndex = unrepaired.indexOf(strategy);
+                if (unrepairedIndex > 0)
+                {
+                    return Collections.singletonList(locations.get(unrepairedIndex).location.getAbsolutePath());
+                }
+                int repairedIndex = repaired.indexOf(strategy);
+                if (repairedIndex > 0)
+                {
+                    return Collections.singletonList(locations.get(repairedIndex).location.getAbsolutePath());
+                }
             }
-            int repairedIndex = repaired.indexOf(strategy);
-            if (repairedIndex > 0)
+            List<String> folders = new ArrayList<>(locations.size());
+            for (Directories.DataDirectory location : locations)
             {
-                return Collections.singletonList(locations.get(repairedIndex).location.getAbsolutePath());
+                folders.add(location.location.getAbsolutePath());
             }
+            return folders;
         }
-        List<String> folders = new ArrayList<>(locations.size());
-        for (Directories.DataDirectory location : locations)
+        finally
         {
-            folders.add(location.location.getAbsolutePath());
+            readLock.unlock();
         }
-        return folders;
+
     }
 
     public boolean supportsEarlyOpen()
     {
-        return repaired.get(0).supportsEarlyOpen();
+        return supportsEarlyOpen;
     }
 }


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


[3/3] cassandra git commit: Merge branch 'cassandra-3.11' into trunk

Posted by pa...@apache.org.
Merge branch 'cassandra-3.11' into trunk


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

Branch: refs/heads/trunk
Commit: 7a29d22e60b52c38bb81a90b854017242850bef4
Parents: ee907a3 fe0ee85
Author: Paulo Motta <pa...@apache.org>
Authored: Wed Jan 10 18:40:00 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Wed Jan 10 18:42:54 2018 -0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../compaction/CompactionStrategyManager.java   | 123 ++++++++++++++-----
 2 files changed, 92 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7a29d22e/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 62559cb,b89ad99..d59fbe3
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,184 -1,9 +1,185 @@@
 +4.0
 + * Make PartitionUpdate and Mutation immutable (CASSANDRA-13867)
 + * Fix CommitLogReplayer exception for CDC data (CASSANDRA-14066)
 + * Fix cassandra-stress startup failure (CASSANDRA-14106)
 + * Remove initialDirectories from CFS (CASSANDRA-13928)
 + * Fix trivial log format error (CASSANDRA-14015)
 + * Allow sstabledump to do a json object per partition (CASSANDRA-13848)
 + * Add option to optimise merkle tree comparison across replicas (CASSANDRA-3200)
 + * Remove unused and deprecated methods from AbstractCompactionStrategy (CASSANDRA-14081)
 + * Fix Distribution.average in cassandra-stress (CASSANDRA-14090)
 + * Support a means of logging all queries as they were invoked (CASSANDRA-13983)
 + * Presize collections (CASSANDRA-13760)
 + * Add GroupCommitLogService (CASSANDRA-13530)
 + * Parallelize initial materialized view build (CASSANDRA-12245)
 + * Fix flaky SecondaryIndexManagerTest.assert[Not]MarkedAsBuilt (CASSANDRA-13965)
 + * Make LWTs send resultset metadata on every request (CASSANDRA-13992)
 + * Fix flaky indexWithFailedInitializationIsNotQueryableAfterPartialRebuild (CASSANDRA-13963)
 + * Introduce leaf-only iterator (CASSANDRA-9988)
 + * Upgrade Guava to 23.3 and Airline to 0.8 (CASSANDRA-13997)
 + * Allow only one concurrent call to StatusLogger (CASSANDRA-12182)
 + * Refactoring to specialised functional interfaces (CASSANDRA-13982)
 + * Speculative retry should allow more friendly params (CASSANDRA-13876)
 + * Throw exception if we send/receive repair messages to incompatible nodes (CASSANDRA-13944)
 + * Replace usages of MessageDigest with Guava's Hasher (CASSANDRA-13291)
 + * Add nodetool cmd to print hinted handoff window (CASSANDRA-13728)
 + * Fix some alerts raised by static analysis (CASSANDRA-13799)
 + * Checksum sstable metadata (CASSANDRA-13321, CASSANDRA-13593)
 + * Add result set metadata to prepared statement MD5 hash calculation (CASSANDRA-10786)
 + * Refactor GcCompactionTest to avoid boxing (CASSANDRA-13941)
 + * Expose recent histograms in JmxHistograms (CASSANDRA-13642)
 + * Fix buffer length comparison when decompressing in netty-based streaming (CASSANDRA-13899)
 + * Properly close StreamCompressionInputStream to release any ByteBuf (CASSANDRA-13906)
 + * Add SERIAL and LOCAL_SERIAL support for cassandra-stress (CASSANDRA-13925)
 + * LCS needlessly checks for L0 STCS candidates multiple times (CASSANDRA-12961)
 + * Correctly close netty channels when a stream session ends (CASSANDRA-13905)
 + * Update lz4 to 1.4.0 (CASSANDRA-13741)
 + * Optimize Paxos prepare and propose stage for local requests (CASSANDRA-13862)
 + * Throttle base partitions during MV repair streaming to prevent OOM (CASSANDRA-13299)
 + * Use compaction threshold for STCS in L0 (CASSANDRA-13861)
 + * Fix problem with min_compress_ratio: 1 and disallow ratio < 1 (CASSANDRA-13703)
 + * Add extra information to SASI timeout exception (CASSANDRA-13677)
 + * Add incremental repair support for --hosts, --force, and subrange repair (CASSANDRA-13818)
 + * Rework CompactionStrategyManager.getScanners synchronization (CASSANDRA-13786)
 + * Add additional unit tests for batch behavior, TTLs, Timestamps (CASSANDRA-13846)
 + * Add keyspace and table name in schema validation exception (CASSANDRA-13845)
 + * Emit metrics whenever we hit tombstone failures and warn thresholds (CASSANDRA-13771)
 + * Make netty EventLoopGroups daemon threads (CASSANDRA-13837)
 + * Race condition when closing stream sessions (CASSANDRA-13852)
 + * NettyFactoryTest is failing in trunk on macOS (CASSANDRA-13831)
 + * Allow changing log levels via nodetool for related classes (CASSANDRA-12696)
 + * Add stress profile yaml with LWT (CASSANDRA-7960)
 + * Reduce memory copies and object creations when acting on ByteBufs (CASSANDRA-13789)
 + * Simplify mx4j configuration (Cassandra-13578)
 + * Fix trigger example on 4.0 (CASSANDRA-13796)
 + * Force minumum timeout value (CASSANDRA-9375)
 + * Use netty for streaming (CASSANDRA-12229)
 + * Use netty for internode messaging (CASSANDRA-8457)
 + * Add bytes repaired/unrepaired to nodetool tablestats (CASSANDRA-13774)
 + * Don't delete incremental repair sessions if they still have sstables (CASSANDRA-13758)
 + * Fix pending repair manager index out of bounds check (CASSANDRA-13769)
 + * Don't use RangeFetchMapCalculator when RF=1 (CASSANDRA-13576)
 + * Don't optimise trivial ranges in RangeFetchMapCalculator (CASSANDRA-13664)
 + * Use an ExecutorService for repair commands instead of new Thread(..).start() (CASSANDRA-13594)
 + * Fix race / ref leak in anticompaction (CASSANDRA-13688)
 + * Expose tasks queue length via JMX (CASSANDRA-12758)
 + * Fix race / ref leak in PendingRepairManager (CASSANDRA-13751)
 + * Enable ppc64le runtime as unsupported architecture (CASSANDRA-13615)
 + * Improve sstablemetadata output (CASSANDRA-11483)
 + * Support for migrating legacy users to roles has been dropped (CASSANDRA-13371)
 + * Introduce error metrics for repair (CASSANDRA-13387)
 + * Refactoring to primitive functional interfaces in AuthCache (CASSANDRA-13732)
 + * Update metrics to 3.1.5 (CASSANDRA-13648)
 + * batch_size_warn_threshold_in_kb can now be set at runtime (CASSANDRA-13699)
 + * Avoid always rebuilding secondary indexes at startup (CASSANDRA-13725)
 + * Upgrade JMH from 1.13 to 1.19 (CASSANDRA-13727)
 + * Upgrade SLF4J from 1.7.7 to 1.7.25 (CASSANDRA-12996)
 + * Default for start_native_transport now true if not set in config (CASSANDRA-13656)
 + * Don't add localhost to the graph when calculating where to stream from (CASSANDRA-13583)
 + * Make CDC availability more deterministic via hard-linking (CASSANDRA-12148)
 + * Allow skipping equality-restricted clustering columns in ORDER BY clause (CASSANDRA-10271)
 + * Use common nowInSec for validation compactions (CASSANDRA-13671)
 + * Improve handling of IR prepare failures (CASSANDRA-13672)
 + * Send IR coordinator messages synchronously (CASSANDRA-13673)
 + * Flush system.repair table before IR finalize promise (CASSANDRA-13660)
 + * Fix column filter creation for wildcard queries (CASSANDRA-13650)
 + * Add 'nodetool getbatchlogreplaythrottle' and 'nodetool setbatchlogreplaythrottle' (CASSANDRA-13614)
 + * fix race condition in PendingRepairManager (CASSANDRA-13659)
 + * Allow noop incremental repair state transitions (CASSANDRA-13658)
 + * Run repair with down replicas (CASSANDRA-10446)
 + * Added started & completed repair metrics (CASSANDRA-13598)
 + * Added started & completed repair metrics (CASSANDRA-13598)
 + * Improve secondary index (re)build failure and concurrency handling (CASSANDRA-10130)
 + * Improve calculation of available disk space for compaction (CASSANDRA-13068)
 + * Change the accessibility of RowCacheSerializer for third party row cache plugins (CASSANDRA-13579)
 + * Allow sub-range repairs for a preview of repaired data (CASSANDRA-13570)
 + * NPE in IR cleanup when columnfamily has no sstables (CASSANDRA-13585)
 + * Fix Randomness of stress values (CASSANDRA-12744)
 + * Allow selecting Map values and Set elements (CASSANDRA-7396)
 + * Fast and garbage-free Streaming Histogram (CASSANDRA-13444)
 + * Update repairTime for keyspaces on completion (CASSANDRA-13539)
 + * Add configurable upper bound for validation executor threads (CASSANDRA-13521)
 + * Bring back maxHintTTL propery (CASSANDRA-12982)
 + * Add testing guidelines (CASSANDRA-13497)
 + * Add more repair metrics (CASSANDRA-13531)
 + * RangeStreamer should be smarter when picking endpoints for streaming (CASSANDRA-4650)
 + * Avoid rewrapping an exception thrown for cache load functions (CASSANDRA-13367)
 + * Log time elapsed for each incremental repair phase (CASSANDRA-13498)
 + * Add multiple table operation support to cassandra-stress (CASSANDRA-8780)
 + * Fix incorrect cqlsh results when selecting same columns multiple times (CASSANDRA-13262)
 + * Fix WriteResponseHandlerTest is sensitive to test execution order (CASSANDRA-13421)
 + * Improve incremental repair logging (CASSANDRA-13468)
 + * Start compaction when incremental repair finishes (CASSANDRA-13454)
 + * Add repair streaming preview (CASSANDRA-13257)
 + * Cleanup isIncremental/repairedAt usage (CASSANDRA-13430)
 + * Change protocol to allow sending key space independent of query string (CASSANDRA-10145)
 + * Make gc_log and gc_warn settable at runtime (CASSANDRA-12661)
 + * Take number of files in L0 in account when estimating remaining compaction tasks (CASSANDRA-13354)
 + * Skip building views during base table streams on range movements (CASSANDRA-13065)
 + * Improve error messages for +/- operations on maps and tuples (CASSANDRA-13197)
 + * Remove deprecated repair JMX APIs (CASSANDRA-11530)
 + * Fix version check to enable streaming keep-alive (CASSANDRA-12929)
 + * Make it possible to monitor an ideal consistency level separate from actual consistency level (CASSANDRA-13289)
 + * Outbound TCP connections ignore internode authenticator (CASSANDRA-13324)
 + * Upgrade junit from 4.6 to 4.12 (CASSANDRA-13360)
 + * Cleanup ParentRepairSession after repairs (CASSANDRA-13359)
 + * Upgrade snappy-java to 1.1.2.6 (CASSANDRA-13336)
 + * Incremental repair not streaming correct sstables (CASSANDRA-13328)
 + * Upgrade the jna version to 4.3.0 (CASSANDRA-13300)
 + * Add the currentTimestamp, currentDate, currentTime and currentTimeUUID functions (CASSANDRA-13132)
 + * Remove config option index_interval (CASSANDRA-10671)
 + * Reduce lock contention for collection types and serializers (CASSANDRA-13271)
 + * Make it possible to override MessagingService.Verb ids (CASSANDRA-13283)
 + * Avoid synchronized on prepareForRepair in ActiveRepairService (CASSANDRA-9292)
 + * Adds the ability to use uncompressed chunks in compressed files (CASSANDRA-10520)
 + * Don't flush sstables when streaming for incremental repair (CASSANDRA-13226)
 + * Remove unused method (CASSANDRA-13227)
 + * Fix minor bugs related to #9143 (CASSANDRA-13217)
 + * Output warning if user increases RF (CASSANDRA-13079)
 + * Remove pre-3.0 streaming compatibility code for 4.0 (CASSANDRA-13081)
 + * Add support for + and - operations on dates (CASSANDRA-11936)
 + * Fix consistency of incrementally repaired data (CASSANDRA-9143)
 + * Increase commitlog version (CASSANDRA-13161)
 + * Make TableMetadata immutable, optimize Schema (CASSANDRA-9425)
 + * Refactor ColumnCondition (CASSANDRA-12981)
 + * Parallelize streaming of different keyspaces (CASSANDRA-4663)
 + * Improved compactions metrics (CASSANDRA-13015)
 + * Speed-up start-up sequence by avoiding un-needed flushes (CASSANDRA-13031)
 + * 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)
 + * Use new token allocation for non bootstrap case as well (CASSANDRA-13080)
 + * Avoid byte-array copy when key cache is disabled (CASSANDRA-13084)
 + * Require forceful decommission if number of nodes is less than replication factor (CASSANDRA-12510)
 + * Allow IN restrictions on column families with collections (CASSANDRA-12654)
 + * 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)
 + * Expose time spent waiting in thread pool queue (CASSANDRA-8398)
 + * Conditionally update index built status to avoid unnecessary flushes (CASSANDRA-12969)
 + * cqlsh auto completion: refactor definition of compaction strategy options (CASSANDRA-12946)
 + * Add support for arithmetic operators (CASSANDRA-11935)
 + * Add histogram for delay to deliver hints (CASSANDRA-13234)
 + * Fix cqlsh automatic protocol downgrade regression (CASSANDRA-13307)
 + * Changing `max_hint_window_in_ms` at runtime (CASSANDRA-11720)
 + * Trivial format error in StorageProxy (CASSANDRA-13551)
 + * Nodetool repair can hang forever if we lose the notification for the repair completing/failing (CASSANDRA-13480)
 + * Anticompaction can cause noisy log messages (CASSANDRA-13684)
 + * Switch to client init for sstabledump (CASSANDRA-13683)
 + * CQLSH: Don't pause when capturing data (CASSANDRA-13743)
 + * nodetool clearsnapshot requires --all to clear all snapshots (CASSANDRA-13391)
 +
 +
  3.11.2
+  * Acquire read lock before accessing CompactionStrategyManager fields (CASSANDRA-14139)
 - * Split CommitLogStressTest to avoid timeout (CASSANDRA-14143)
   * Avoid invalidating disk boundaries unnecessarily (CASSANDRA-14083)
   * Avoid exposing compaction strategy index externally (CASSANDRA-14082)
 - * Prevent continuous schema exchange between 3.0 and 3.11 nodes (CASSANDRA-14109)
   * Fix imbalanced disks when replacing node with same address with JBOD (CASSANDRA-14084)
   * Reload compaction strategies when disk boundaries are invalidated (CASSANDRA-13948)
   * Remove OpenJDK log warning (CASSANDRA-13916)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7a29d22e/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
index 13d90a0,a50f428..7ad6e91
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
@@@ -28,9 -27,11 +28,10 @@@ import java.util.stream.Stream
  
  import com.google.common.annotations.VisibleForTesting;
  import com.google.common.collect.Iterables;
++import com.google.common.collect.Lists;
  
  import org.apache.cassandra.db.DiskBoundaries;
 -import org.apache.cassandra.db.Memtable;
  import org.apache.cassandra.index.Index;
 -import com.google.common.primitives.Ints;
  
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
@@@ -104,12 -102,12 +105,12 @@@ public class CompactionStrategyManager 
  
          If a user changes the local compaction strategy and then later ALTERs a compaction parameter,
          we will use the new compaction parameters.
 -     **/
 +     */
      private volatile CompactionParams schemaCompactionParams;
      private boolean shouldDefragment;
+     private boolean supportsEarlyOpen;
      private int fanout;
  
- 
      public CompactionStrategyManager(ColumnFamilyStore cfs)
      {
          this(cfs, cfs::getDiskBoundaries, cfs.getPartitioner().splitter().isPresent());
@@@ -255,8 -216,8 +256,9 @@@
              }
              repaired.forEach(AbstractCompactionStrategy::startup);
              unrepaired.forEach(AbstractCompactionStrategy::startup);
 +            pendingRepairs.forEach(PendingRepairManager::startup);
              shouldDefragment = repaired.get(0).shouldDefragment();
+             supportsEarlyOpen = repaired.get(0).supportsEarlyOpen();
              fanout = (repaired.get(0) instanceof LeveledCompactionStrategy) ? ((LeveledCompactionStrategy) repaired.get(0)).getLevelFanoutSize() : LeveledCompactionStrategy.DEFAULT_LEVEL_FANOUT_SIZE;
          }
          finally
@@@ -335,39 -293,6 +337,79 @@@
          }
      }
  
 +    @VisibleForTesting
 +    List<AbstractCompactionStrategy> getRepaired()
 +    {
-         return repaired;
++        readLock.lock();
++        try
++        {
++            return Lists.newArrayList(repaired);
++        }
++        finally
++        {
++            readLock.unlock();
++        }
 +    }
 +
 +    @VisibleForTesting
 +    List<AbstractCompactionStrategy> getUnrepaired()
 +    {
-         return unrepaired;
++        readLock.lock();
++        try
++        {
++            return Lists.newArrayList(unrepaired);
++        }
++        finally
++        {
++            readLock.unlock();
++        }
 +    }
 +
 +    @VisibleForTesting
 +    List<AbstractCompactionStrategy> getForPendingRepair(UUID sessionID)
 +    {
-         List<AbstractCompactionStrategy> strategies = new ArrayList<>(pendingRepairs.size());
-         pendingRepairs.forEach(p -> strategies.add(p.get(sessionID)));
-         return strategies;
++        readLock.lock();
++        try
++        {
++            List<AbstractCompactionStrategy> strategies = new ArrayList<>(pendingRepairs.size());
++            pendingRepairs.forEach(p -> strategies.add(p.get(sessionID)));
++            return strategies;
++        }
++        finally
++        {
++            readLock.unlock();
++        }
 +    }
 +
 +    @VisibleForTesting
 +    Set<UUID> pendingRepairs()
 +    {
-         Set<UUID> ids = new HashSet<>();
-         pendingRepairs.forEach(p -> ids.addAll(p.getSessions()));
-         return ids;
++        readLock.lock();
++        try
++        {
++            Set<UUID> ids = new HashSet<>();
++            pendingRepairs.forEach(p -> ids.addAll(p.getSessions()));
++            return ids;
++        }
++        finally
++        {
++            readLock.unlock();
++        }
 +    }
 +
 +    public boolean hasDataForPendingRepair(UUID sessionID)
 +    {
-         return Iterables.any(pendingRepairs, prm -> prm.hasDataForSession(sessionID));
++        readLock.lock();
++        try
++        {
++            return Iterables.any(pendingRepairs, prm -> prm.hasDataForSession(sessionID));
++        }
++        finally
++        {
++            readLock.unlock();
++        }
 +    }
 +
      public void shutdown()
      {
          writeLock.lock();
@@@ -1161,77 -1052,39 +1203,94 @@@
  
      public List<String> getStrategyFolders(AbstractCompactionStrategy strategy)
      {
-         Directories.DataDirectory[] locations = cfs.getDirectories().getWriteableLocations();
-         if (partitionSSTablesByTokenRange)
+         readLock.lock();
+         try
          {
-             int unrepairedIndex = unrepaired.indexOf(strategy);
-             if (unrepairedIndex > 0)
-             {
-                 return Collections.singletonList(locations[unrepairedIndex].location.getAbsolutePath());
-             }
-             int repairedIndex = repaired.indexOf(strategy);
-             if (repairedIndex > 0)
-             {
-                 return Collections.singletonList(locations[repairedIndex].location.getAbsolutePath());
-             }
-             for (int i = 0; i < pendingRepairs.size(); i++)
 -            List<Directories.DataDirectory> locations = currentBoundaries.directories;
++            Directories.DataDirectory[] locations = cfs.getDirectories().getWriteableLocations();
+             if (partitionSSTablesByTokenRange)
              {
-                 PendingRepairManager pending = pendingRepairs.get(i);
-                 if (pending.hasStrategy(strategy))
+                 int unrepairedIndex = unrepaired.indexOf(strategy);
+                 if (unrepairedIndex > 0)
+                 {
 -                    return Collections.singletonList(locations.get(unrepairedIndex).location.getAbsolutePath());
++                    return Collections.singletonList(locations[unrepairedIndex].location.getAbsolutePath());
+                 }
+                 int repairedIndex = repaired.indexOf(strategy);
+                 if (repairedIndex > 0)
                  {
-                     return Collections.singletonList(locations[i].location.getAbsolutePath());
 -                    return Collections.singletonList(locations.get(repairedIndex).location.getAbsolutePath());
++                    return Collections.singletonList(locations[repairedIndex].location.getAbsolutePath());
 +                }
++                for (int i = 0; i < pendingRepairs.size(); i++)
++                {
++                    PendingRepairManager pending = pendingRepairs.get(i);
++                    if (pending.hasStrategy(strategy))
++                    {
++                        return Collections.singletonList(locations[i].location.getAbsolutePath());
++                    }
+                 }
+             }
 -            List<String> folders = new ArrayList<>(locations.size());
++            List<String> folders = new ArrayList<>(locations.length);
+             for (Directories.DataDirectory location : locations)
+             {
+                 folders.add(location.location.getAbsolutePath());
              }
+             return folders;
          }
-         List<String> folders = new ArrayList<>(locations.length);
-         for (Directories.DataDirectory location : locations)
+         finally
          {
-             folders.add(location.location.getAbsolutePath());
+             readLock.unlock();
          }
-         return folders;
 -
      }
  
      public boolean supportsEarlyOpen()
      {
-         return repaired.get(0).supportsEarlyOpen();
+         return supportsEarlyOpen;
      }
 +
 +    @VisibleForTesting
 +    List<PendingRepairManager> getPendingRepairManagers()
 +    {
-         return pendingRepairs;
++        maybeReloadDiskBoundaries();
++        readLock.lock();
++        try
++        {
++            return pendingRepairs;
++        }
++        finally
++        {
++            readLock.unlock();
++        }
 +    }
 +
 +    /**
 +     * Mutates sstable repairedAt times and notifies listeners of the change with the writeLock held. Prevents races
 +     * with other processes between when the metadata is changed and when sstables are moved between strategies.
 +     */
 +    public void mutateRepaired(Collection<SSTableReader> sstables, long repairedAt, UUID pendingRepair) throws IOException
 +    {
 +        Set<SSTableReader> changed = new HashSet<>();
 +
 +        writeLock.lock();
 +        try
 +        {
 +            for (SSTableReader sstable: sstables)
 +            {
 +                sstable.descriptor.getMetadataSerializer().mutateRepaired(sstable.descriptor, repairedAt, pendingRepair);
 +                sstable.reloadSSTableMetadata();
 +                changed.add(sstable);
 +            }
 +        }
 +        finally
 +        {
 +            try
 +            {
 +                // if there was an exception mutating repairedAt, we should still notify for the
 +                // sstables that we were able to modify successfully before releasing the lock
 +                cfs.getTracker().notifySSTableRepairedStatusChanged(changed);
 +            }
 +            finally
 +            {
 +                writeLock.unlock();
 +            }
 +        }
 +    }
  }


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