You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sa...@apache.org on 2016/07/06 11:17:57 UTC

[1/6] cassandra git commit: Avoid potential race when rebuilding CFMetaData.columnMetadata

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 73c71672b -> dd05e46f0
  refs/heads/cassandra-3.9 262cd38b0 -> 5ad17634a
  refs/heads/trunk 5fae533f3 -> a4e738777


Avoid potential race when rebuilding CFMetaData.columnMetadata

Patch by Sam Tunnicliffe; reviewed by Aleksey Yeschenko for
CASSANDRA-12098


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

Branch: refs/heads/cassandra-3.0
Commit: dd05e46f0cb5475edf72676230474e0ad0f9cdbf
Parents: 73c7167
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Tue Jul 5 13:02:20 2016 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Wed Jul 6 12:08:22 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                         |  1 +
 .../org/apache/cassandra/config/CFMetaData.java     | 16 +++++++---------
 2 files changed, 8 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd05e46f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b3063b4..02786c5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
  * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)
  * Always select the live sstables when getting sstables in bounds (CASSANDRA-11944)
  * Fix column ordering of results with static columns for Thrift requests in

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd05e46f/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index e263697..5678ada 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -22,7 +22,6 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
@@ -53,9 +52,6 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.utils.*;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.UUIDGen;
 import org.github.jamm.Unmetered;
 
 /**
@@ -109,7 +105,7 @@ public final class CFMetaData
      * clustering key ones, those list are ordered by the "component index" of the
      * elements.
      */
-    private final Map<ByteBuffer, ColumnDefinition> columnMetadata = new ConcurrentHashMap<>(); // not on any hot path
+    private volatile Map<ByteBuffer, ColumnDefinition> columnMetadata = new HashMap<>();
     private volatile List<ColumnDefinition> partitionKeyColumns;  // Always of size keyValidator.componentsCount, null padded if necessary
     private volatile List<ColumnDefinition> clusteringColumns;    // Of size comparator.componentsCount or comparator.componentsCount -1, null padded if necessary
     private volatile PartitionColumns partitionColumns;           // Always non-PK, non-clustering columns
@@ -298,16 +294,18 @@ public final class CFMetaData
     {
         this.comparator = new ClusteringComparator(extractTypes(clusteringColumns));
 
-        this.columnMetadata.clear();
+        Map<ByteBuffer, ColumnDefinition> newColumnMetadata = new HashMap<>();
         for (ColumnDefinition def : partitionKeyColumns)
-            this.columnMetadata.put(def.name.bytes, def);
+            newColumnMetadata.put(def.name.bytes, def);
         for (ColumnDefinition def : clusteringColumns)
         {
-            this.columnMetadata.put(def.name.bytes, def);
+            newColumnMetadata.put(def.name.bytes, def);
             def.type.checkComparable();
         }
         for (ColumnDefinition def : partitionColumns)
-            this.columnMetadata.put(def.name.bytes, def);
+            newColumnMetadata.put(def.name.bytes, def);
+
+        this.columnMetadata = newColumnMetadata;
 
         List<AbstractType<?>> keyTypes = extractTypes(partitionKeyColumns);
         this.keyValidator = keyTypes.size() == 1 ? keyTypes.get(0) : CompositeType.getInstance(keyTypes);


[6/6] cassandra git commit: Merge branch 'cassandra-3.9' into trunk

Posted by sa...@apache.org.
Merge branch 'cassandra-3.9' into trunk


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

Branch: refs/heads/trunk
Commit: a4e7387772d5e57d737c8a780d552263ed51bd4a
Parents: 5fae533 5ad1763
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Wed Jul 6 12:12:21 2016 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Wed Jul 6 12:12:21 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                         |  1 +
 .../org/apache/cassandra/config/CFMetaData.java     | 16 +++++++---------
 2 files changed, 8 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


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


[2/6] cassandra git commit: Avoid potential race when rebuilding CFMetaData.columnMetadata

Posted by sa...@apache.org.
Avoid potential race when rebuilding CFMetaData.columnMetadata

Patch by Sam Tunnicliffe; reviewed by Aleksey Yeschenko for
CASSANDRA-12098


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

Branch: refs/heads/cassandra-3.9
Commit: dd05e46f0cb5475edf72676230474e0ad0f9cdbf
Parents: 73c7167
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Tue Jul 5 13:02:20 2016 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Wed Jul 6 12:08:22 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                         |  1 +
 .../org/apache/cassandra/config/CFMetaData.java     | 16 +++++++---------
 2 files changed, 8 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd05e46f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b3063b4..02786c5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
  * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)
  * Always select the live sstables when getting sstables in bounds (CASSANDRA-11944)
  * Fix column ordering of results with static columns for Thrift requests in

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd05e46f/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index e263697..5678ada 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -22,7 +22,6 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
@@ -53,9 +52,6 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.utils.*;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.UUIDGen;
 import org.github.jamm.Unmetered;
 
 /**
@@ -109,7 +105,7 @@ public final class CFMetaData
      * clustering key ones, those list are ordered by the "component index" of the
      * elements.
      */
-    private final Map<ByteBuffer, ColumnDefinition> columnMetadata = new ConcurrentHashMap<>(); // not on any hot path
+    private volatile Map<ByteBuffer, ColumnDefinition> columnMetadata = new HashMap<>();
     private volatile List<ColumnDefinition> partitionKeyColumns;  // Always of size keyValidator.componentsCount, null padded if necessary
     private volatile List<ColumnDefinition> clusteringColumns;    // Of size comparator.componentsCount or comparator.componentsCount -1, null padded if necessary
     private volatile PartitionColumns partitionColumns;           // Always non-PK, non-clustering columns
@@ -298,16 +294,18 @@ public final class CFMetaData
     {
         this.comparator = new ClusteringComparator(extractTypes(clusteringColumns));
 
-        this.columnMetadata.clear();
+        Map<ByteBuffer, ColumnDefinition> newColumnMetadata = new HashMap<>();
         for (ColumnDefinition def : partitionKeyColumns)
-            this.columnMetadata.put(def.name.bytes, def);
+            newColumnMetadata.put(def.name.bytes, def);
         for (ColumnDefinition def : clusteringColumns)
         {
-            this.columnMetadata.put(def.name.bytes, def);
+            newColumnMetadata.put(def.name.bytes, def);
             def.type.checkComparable();
         }
         for (ColumnDefinition def : partitionColumns)
-            this.columnMetadata.put(def.name.bytes, def);
+            newColumnMetadata.put(def.name.bytes, def);
+
+        this.columnMetadata = newColumnMetadata;
 
         List<AbstractType<?>> keyTypes = extractTypes(partitionKeyColumns);
         this.keyValidator = keyTypes.size() == 1 ? keyTypes.get(0) : CompositeType.getInstance(keyTypes);


[3/6] cassandra git commit: Avoid potential race when rebuilding CFMetaData.columnMetadata

Posted by sa...@apache.org.
Avoid potential race when rebuilding CFMetaData.columnMetadata

Patch by Sam Tunnicliffe; reviewed by Aleksey Yeschenko for
CASSANDRA-12098


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

Branch: refs/heads/trunk
Commit: dd05e46f0cb5475edf72676230474e0ad0f9cdbf
Parents: 73c7167
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Tue Jul 5 13:02:20 2016 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Wed Jul 6 12:08:22 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                         |  1 +
 .../org/apache/cassandra/config/CFMetaData.java     | 16 +++++++---------
 2 files changed, 8 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd05e46f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b3063b4..02786c5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
  * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)
  * Always select the live sstables when getting sstables in bounds (CASSANDRA-11944)
  * Fix column ordering of results with static columns for Thrift requests in

http://git-wip-us.apache.org/repos/asf/cassandra/blob/dd05e46f/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index e263697..5678ada 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -22,7 +22,6 @@ import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
@@ -53,9 +52,6 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.utils.*;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.UUIDGen;
 import org.github.jamm.Unmetered;
 
 /**
@@ -109,7 +105,7 @@ public final class CFMetaData
      * clustering key ones, those list are ordered by the "component index" of the
      * elements.
      */
-    private final Map<ByteBuffer, ColumnDefinition> columnMetadata = new ConcurrentHashMap<>(); // not on any hot path
+    private volatile Map<ByteBuffer, ColumnDefinition> columnMetadata = new HashMap<>();
     private volatile List<ColumnDefinition> partitionKeyColumns;  // Always of size keyValidator.componentsCount, null padded if necessary
     private volatile List<ColumnDefinition> clusteringColumns;    // Of size comparator.componentsCount or comparator.componentsCount -1, null padded if necessary
     private volatile PartitionColumns partitionColumns;           // Always non-PK, non-clustering columns
@@ -298,16 +294,18 @@ public final class CFMetaData
     {
         this.comparator = new ClusteringComparator(extractTypes(clusteringColumns));
 
-        this.columnMetadata.clear();
+        Map<ByteBuffer, ColumnDefinition> newColumnMetadata = new HashMap<>();
         for (ColumnDefinition def : partitionKeyColumns)
-            this.columnMetadata.put(def.name.bytes, def);
+            newColumnMetadata.put(def.name.bytes, def);
         for (ColumnDefinition def : clusteringColumns)
         {
-            this.columnMetadata.put(def.name.bytes, def);
+            newColumnMetadata.put(def.name.bytes, def);
             def.type.checkComparable();
         }
         for (ColumnDefinition def : partitionColumns)
-            this.columnMetadata.put(def.name.bytes, def);
+            newColumnMetadata.put(def.name.bytes, def);
+
+        this.columnMetadata = newColumnMetadata;
 
         List<AbstractType<?>> keyTypes = extractTypes(partitionKeyColumns);
         this.keyValidator = keyTypes.size() == 1 ? keyTypes.get(0) : CompositeType.getInstance(keyTypes);


[5/6] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by sa...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9


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

Branch: refs/heads/cassandra-3.9
Commit: 5ad17634a1fb75d09ab5a6cb234d7460fed0c3e0
Parents: 262cd38 dd05e46
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Wed Jul 6 12:10:41 2016 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Wed Jul 6 12:10:41 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                         |  1 +
 .../org/apache/cassandra/config/CFMetaData.java     | 16 +++++++---------
 2 files changed, 8 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5ad17634/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index afeef2c,02786c5..2861cf7
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,7 -1,5 +1,8 @@@
 -3.0.9
 +3.9
 + * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
 + * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 +Merged from 3.0:
+  * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
   * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)
   * Always select the live sstables when getting sstables in bounds (CASSANDRA-11944)
   * Fix column ordering of results with static columns for Thrift requests in

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5ad17634/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/CFMetaData.java
index caa2059,5678ada..4708033
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@@ -22,10 -22,8 +22,9 @@@ import java.lang.reflect.Constructor
  import java.lang.reflect.InvocationTargetException;
  import java.nio.ByteBuffer;
  import java.util.*;
- import java.util.concurrent.ConcurrentHashMap;
  import java.util.concurrent.ThreadLocalRandom;
  import java.util.concurrent.TimeUnit;
 +import java.util.regex.Pattern;
  import java.util.stream.Collectors;
  
  import com.google.common.annotations.VisibleForTesting;


[4/6] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.9

Posted by sa...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.9


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

Branch: refs/heads/trunk
Commit: 5ad17634a1fb75d09ab5a6cb234d7460fed0c3e0
Parents: 262cd38 dd05e46
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Wed Jul 6 12:10:41 2016 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Wed Jul 6 12:10:41 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                         |  1 +
 .../org/apache/cassandra/config/CFMetaData.java     | 16 +++++++---------
 2 files changed, 8 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5ad17634/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index afeef2c,02786c5..2861cf7
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,7 -1,5 +1,8 @@@
 -3.0.9
 +3.9
 + * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
 + * Increase size of flushExecutor thread pool (CASSANDRA-12071)
 +Merged from 3.0:
+  * Avoid potential race when rebuilding CFMetaData (CASSANDRA-12098)
   * Avoid missing sstables when getting the canonical sstables (CASSANDRA-11996)
   * Always select the live sstables when getting sstables in bounds (CASSANDRA-11944)
   * Fix column ordering of results with static columns for Thrift requests in

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5ad17634/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/CFMetaData.java
index caa2059,5678ada..4708033
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@@ -22,10 -22,8 +22,9 @@@ import java.lang.reflect.Constructor
  import java.lang.reflect.InvocationTargetException;
  import java.nio.ByteBuffer;
  import java.util.*;
- import java.util.concurrent.ConcurrentHashMap;
  import java.util.concurrent.ThreadLocalRandom;
  import java.util.concurrent.TimeUnit;
 +import java.util.regex.Pattern;
  import java.util.stream.Collectors;
  
  import com.google.common.annotations.VisibleForTesting;