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:59 UTC

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

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);