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

[01/16] cassandra git commit: Fix nodetool status logic to detect if their is more than 1 user keyspace

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 393c6ec4f -> f4b21f5ff
  refs/heads/cassandra-2.2 f3b5ff842 -> f7c75857d
  refs/heads/cassandra-3.0 9cfbc31bc -> 931d2e5f3
  refs/heads/trunk 48815d4a1 -> 09fd433cb


Fix nodetool status logic to detect if their is more than 1 user keyspace

explanation

patch by krummas & slebresne; reviewed by krummas & slebresne for CASSANDRA-10176


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

Branch: refs/heads/cassandra-2.1
Commit: f4b21f5ffa962de19901d1841f89f529ccc335b3
Parents: 393c6ec
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 3 21:04:17 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:23:54 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/config/Schema.java     | 22 ++++++++++++++++++--
 .../locator/AbstractReplicationStrategy.java    |  5 +++++
 .../locator/NetworkTopologyStrategy.java        |  6 ++++++
 .../cassandra/service/StorageService.java       | 22 ++++++++++++--------
 5 files changed, 45 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9f51291..9a9c4f7 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.14
+ * Fix incorrect warning in 'nodetool status' (CASSANDRA-10176)
  * Properly release sstable ref when doing offline scrub (CASSANDRA-10697)
  * Improve nodetool status performance for large cluster (CASSANDRA-7238)
  * Make it clear what DTCS timestamp_resolution is used for (CASSANDRA-11041)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/config/Schema.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Schema.java b/src/java/org/apache/cassandra/config/Schema.java
index fada670..739c8ca 100644
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@ -28,12 +28,14 @@ import com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.auth.Auth;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.ConcurrentBiMap;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
@@ -67,6 +69,7 @@ public class Schema
     // 59adb24e-f3cd-3e02-97f0-5b395827453f
     public static final UUID emptyVersion;
     public static final ImmutableSet<String> systemKeyspaceNames = ImmutableSet.of(Keyspace.SYSTEM_KS);
+    public static final ImmutableSet<String> replicatedSystemKeyspaceNames = ImmutableSet.of(Tracing.TRACE_KS, Auth.AUTH_KS);
 
     static
     {
@@ -288,12 +291,27 @@ public class Schema
         return keyspaces.get(keyspaceName);
     }
 
+    private Set<String> getNonSystemKeyspacesSet()
+    {
+        return Sets.difference(keyspaces.keySet(), systemKeyspaceNames);
+    }
+
     /**
-     * @return collection of the non-system keyspaces
+     * @return collection of the non-system keyspaces (note that this count as system only the
+     * non replicated keyspaces, so keyspace like system_traces which are replicated are actually
+     * returned. See getUserKeyspace() below if you don't want those)
      */
     public List<String> getNonSystemKeyspaces()
     {
-        return ImmutableList.copyOf(Sets.difference(keyspaces.keySet(), systemKeyspaceNames));
+        return ImmutableList.copyOf(getNonSystemKeyspacesSet());
+    }
+
+    /**
+     * @return collection of the user defined keyspaces
+     */
+    public List<String> getUserKeyspaces()
+    {
+        return ImmutableList.copyOf(Sets.difference(getNonSystemKeyspacesSet(), replicatedSystemKeyspaceNames));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
index b6599f1..2178e17 100644
--- a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
@@ -303,6 +303,11 @@ public abstract class AbstractReplicationStrategy
         return strategyClass;
     }
 
+    public boolean hasSameSettings(AbstractReplicationStrategy other)
+    {
+        return getClass().equals(other.getClass()) && getReplicationFactor() == other.getReplicationFactor();
+    }
+
     protected void validateReplicationFactor(String rf) throws ConfigurationException
     {
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
index 5e5ecb7..a792aa8 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -199,4 +199,10 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
         // We explicitely allow all options
         return null;
     }
+
+    @Override
+    public boolean hasSameSettings(AbstractReplicationStrategy other)
+    {
+        return super.hasSameSettings(other) && ((NetworkTopologyStrategy) other).datacenters.equals(datacenters);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 02f6cf4..98e2251 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -4032,18 +4032,22 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
         else
         {
-            List<String> nonSystemKeyspaces = Schema.instance.getNonSystemKeyspaces();
+            List<String> userKeyspaces = Schema.instance.getUserKeyspaces();
 
-            //system_traces is a non-system keyspace however it needs to be counted as one for this process
-            int specialTableCount = 0;
-            if (nonSystemKeyspaces.contains("system_traces"))
+            if (userKeyspaces.size() > 0)
             {
-                specialTableCount += 1;
+                keyspace = userKeyspaces.iterator().next();
+                AbstractReplicationStrategy replicationStrategy = Schema.instance.getKeyspaceInstance(keyspace).getReplicationStrategy();
+                for (String keyspaceName : userKeyspaces)
+                {
+                    if (!Schema.instance.getKeyspaceInstance(keyspaceName).getReplicationStrategy().hasSameSettings(replicationStrategy))
+                        throw new IllegalStateException("Non-system keyspaces don't have the same replication settings, effective ownership information is meaningless");
+                }
+            }
+            else
+            {
+                keyspace = "system_traces";
             }
-            if (nonSystemKeyspaces.size() > specialTableCount)
-                throw new IllegalStateException("Non-system keyspaces don't have the same replication settings, effective ownership information is meaningless");
-
-            keyspace = "system_traces";
 
             Keyspace keyspaceInstance = Schema.instance.getKeyspaceInstance(keyspace);
             if (keyspaceInstance == null)


[11/16] cassandra git commit: Add missing CHANGES entry

Posted by sl...@apache.org.
Add missing CHANGES entry


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

Branch: refs/heads/cassandra-3.0
Commit: f7c75857da50a73bfa63f206bdd3573c167b46c4
Parents: 6f5dddd
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 17 11:31:31 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:31:31 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7c75857/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 49bc581..5c912a1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@
  * Fix paging on DISTINCT queries repeats result when first row in partition changes
    (CASSANDRA-10010)
 Merged from 2.1:
+ * Fix incorrect warning in 'nodetool status' (CASSANDRA-10176)
  * Properly release sstable ref when doing offline scrub (CASSANDRA-10697)
  * Improve nodetool status performance for large cluster (CASSANDRA-7238)
  * Gossiper#isEnabled is not thread safe (CASSANDRA-11116)


[09/16] cassandra git commit: Merge commit '6f5dddd' into cassandra-3.0

Posted by sl...@apache.org.
Merge commit '6f5dddd' 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/ce39f81d
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ce39f81d
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ce39f81d

Branch: refs/heads/trunk
Commit: ce39f81d26489ba292447a18a66057980b0c3f53
Parents: 9cfbc31 6f5dddd
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 17 11:25:53 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:27:27 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/config/Schema.java     | 28 ++++++++++++++++++--
 .../locator/AbstractReplicationStrategy.java    |  5 ++++
 .../locator/NetworkTopologyStrategy.java        |  6 +++++
 .../cassandra/service/StorageService.java       | 22 ++++++++-------
 5 files changed, 51 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ce39f81d/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 52fc3ab,49bc581..32c165b
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -22,6 -9,6 +22,7 @@@ Merged from 2.2
   * Fix paging on DISTINCT queries repeats result when first row in partition changes
     (CASSANDRA-10010)
  Merged from 2.1:
++ * Fix incorrect warning in 'nodetool status' (CASSANDRA-10176)
   * Properly release sstable ref when doing offline scrub (CASSANDRA-10697)
   * Improve nodetool status performance for large cluster (CASSANDRA-7238)
   * Gossiper#isEnabled is not thread safe (CASSANDRA-11116)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ce39f81d/src/java/org/apache/cassandra/config/Schema.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/Schema.java
index 7ffa89e,2cd7611..3fd9f11
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@@ -27,18 -27,21 +27,21 @@@ import com.google.common.collect.Sets
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
+ import org.apache.cassandra.auth.AuthKeyspace;
 -import org.apache.cassandra.cql3.functions.Functions;
 -import org.apache.cassandra.cql3.functions.UDAggregate;
 -import org.apache.cassandra.cql3.functions.UDFunction;
 -import org.apache.cassandra.db.*;
 +import org.apache.cassandra.cql3.functions.*;
 +import org.apache.cassandra.db.ColumnFamilyStore;
  import org.apache.cassandra.db.Keyspace;
 +import org.apache.cassandra.db.SystemKeyspace;
  import org.apache.cassandra.db.commitlog.CommitLog;
  import org.apache.cassandra.db.compaction.CompactionManager;
 +import org.apache.cassandra.db.marshal.AbstractType;
  import org.apache.cassandra.db.marshal.UserType;
 -import org.apache.cassandra.db.index.SecondaryIndex;
 +import org.apache.cassandra.index.Index;
  import org.apache.cassandra.io.sstable.Descriptor;
 -import org.apache.cassandra.schema.LegacySchemaTables;
+ import org.apache.cassandra.repair.SystemDistributedKeyspace;
 +import org.apache.cassandra.schema.*;
  import org.apache.cassandra.service.MigrationManager;
+ import org.apache.cassandra.tracing.TraceKeyspace;
  import org.apache.cassandra.utils.ConcurrentBiMap;
  import org.apache.cassandra.utils.Pair;
  import org.cliffc.high_scale_lib.NonBlockingHashMap;
@@@ -49,9 -52,6 +52,14 @@@ public class Schem
  
      public static final Schema instance = new Schema();
  
 +    /* system keyspace names (the ones with LocalStrategy replication strategy) */
 +    public static final Set<String> SYSTEM_KEYSPACE_NAMES = ImmutableSet.of(SystemKeyspace.NAME, SchemaKeyspace.NAME);
 +
++    /* replicate system keyspace names (the ones with a "true" replication strategy) */
++    public static final Set<String> REPLICATED_SYSTEM_KEYSPACE_NAMES = ImmutableSet.of(TraceKeyspace.NAME,
++                                                                                       AuthKeyspace.NAME,
++                                                                                       SystemDistributedKeyspace.NAME);
++
      /**
       * longest permissible KS or CF name.  Our main concern is that filename not be more than 255 characters;
       * the filename will contain both the KS and CF names. Since non-schema-name components only take up
@@@ -74,6 -74,10 +82,7 @@@
      // 59adb24e-f3cd-3e02-97f0-5b395827453f
      public static final UUID emptyVersion;
  
 -    private static final ImmutableSet<String> replicatedSystemKeyspaceNames = ImmutableSet.of(TraceKeyspace.NAME,
 -                                                                                              AuthKeyspace.NAME,
 -                                                                                              SystemDistributedKeyspace.NAME);
+ 
      static
      {
          try
@@@ -323,12 -307,27 +332,27 @@@
          return keyspaces.get(keyspaceName);
      }
  
+     private Set<String> getNonSystemKeyspacesSet()
+     {
 -        return Sets.difference(keyspaces.keySet(), Collections.singleton(SystemKeyspace.NAME));
++        return Sets.difference(keyspaces.keySet(), SYSTEM_KEYSPACE_NAMES);
+     }
+ 
      /**
-      * @return collection of the non-system keyspaces
+      * @return collection of the non-system keyspaces (note that this count as system only the
+      * non replicated keyspaces, so keyspace like system_traces which are replicated are actually
+      * returned. See getUserKeyspace() below if you don't want those)
       */
      public List<String> getNonSystemKeyspaces()
      {
-         return ImmutableList.copyOf(Sets.difference(keyspaces.keySet(), SYSTEM_KEYSPACE_NAMES));
+         return ImmutableList.copyOf(getNonSystemKeyspacesSet());
+     }
+ 
+     /**
+      * @return collection of the user defined keyspaces
+      */
+     public List<String> getUserKeyspaces()
+     {
 -        return ImmutableList.copyOf(Sets.difference(getNonSystemKeyspacesSet(), replicatedSystemKeyspaceNames));
++        return ImmutableList.copyOf(Sets.difference(getNonSystemKeyspacesSet(), REPLICATED_SYSTEM_KEYSPACE_NAMES));
      }
  
      /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ce39f81d/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 7cca516,0e716e8..185c988
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -4157,18 -4045,22 +4157,22 @@@ public class StorageService extends Not
          }
          else
          {
-             List<String> nonSystemKeyspaces = Schema.instance.getNonSystemKeyspaces();
+             List<String> userKeyspaces = Schema.instance.getUserKeyspaces();
  
-             //system_traces is a non-system keyspace however it needs to be counted as one for this process
-             int specialTableCount = 0;
-             if (nonSystemKeyspaces.contains("system_traces"))
+             if (userKeyspaces.size() > 0)
              {
-                 specialTableCount += 1;
 -                keyspace = userKeyspaces.iterator().next();
++                keyspace = userKeyspaces.get(0);
+                 AbstractReplicationStrategy replicationStrategy = Schema.instance.getKeyspaceInstance(keyspace).getReplicationStrategy();
+                 for (String keyspaceName : userKeyspaces)
+                 {
+                     if (!Schema.instance.getKeyspaceInstance(keyspaceName).getReplicationStrategy().hasSameSettings(replicationStrategy))
+                         throw new IllegalStateException("Non-system keyspaces don't have the same replication settings, effective ownership information is meaningless");
+                 }
+             }
+             else
+             {
+                 keyspace = "system_traces";
              }
-             if (nonSystemKeyspaces.size() > specialTableCount)
-                 throw new IllegalStateException("Non-system keyspaces don't have the same replication settings, effective ownership information is meaningless");
- 
-             keyspace = "system_traces";
  
              Keyspace keyspaceInstance = Schema.instance.getKeyspaceInstance(keyspace);
              if (keyspaceInstance == null)


[06/16] cassandra git commit: Merge commit 'f4b21f5' into cassandra-2.2

Posted by sl...@apache.org.
Merge commit 'f4b21f5' 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/6f5dddd9
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6f5dddd9
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6f5dddd9

Branch: refs/heads/trunk
Commit: 6f5dddd966fc16a20f2af8add74b70a9bb11b559
Parents: f3b5ff8 f4b21f5
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 17 11:25:04 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:25:31 2016 +0100

----------------------------------------------------------------------
 .../org/apache/cassandra/config/Schema.java     | 27 ++++++++++++++++++--
 .../locator/AbstractReplicationStrategy.java    |  5 ++++
 .../locator/NetworkTopologyStrategy.java        |  6 +++++
 .../cassandra/service/StorageService.java       | 22 +++++++++-------
 4 files changed, 49 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6f5dddd9/src/java/org/apache/cassandra/config/Schema.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/Schema.java
index 110029e,739c8ca..2cd7611
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@@ -26,19 -28,16 +27,22 @@@ import com.google.common.collect.Sets
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
 -import org.apache.cassandra.auth.Auth;
++import org.apache.cassandra.auth.AuthKeyspace;
 +import org.apache.cassandra.cql3.functions.Functions;
 +import org.apache.cassandra.cql3.functions.UDAggregate;
 +import org.apache.cassandra.cql3.functions.UDFunction;
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.Keyspace;
 +import org.apache.cassandra.db.commitlog.CommitLog;
 +import org.apache.cassandra.db.compaction.CompactionManager;
 +import org.apache.cassandra.db.marshal.UserType;
  import org.apache.cassandra.db.index.SecondaryIndex;
 -import org.apache.cassandra.db.index.SecondaryIndexManager;
  import org.apache.cassandra.io.sstable.Descriptor;
 +import org.apache.cassandra.schema.LegacySchemaTables;
++import org.apache.cassandra.repair.SystemDistributedKeyspace;
  import org.apache.cassandra.service.MigrationManager;
 -import org.apache.cassandra.tracing.Tracing;
++import org.apache.cassandra.tracing.TraceKeyspace;
  import org.apache.cassandra.utils.ConcurrentBiMap;
 -import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.Pair;
  import org.cliffc.high_scale_lib.NonBlockingHashMap;
  
@@@ -69,7 -68,9 +73,11 @@@ public class Schem
  
      // 59adb24e-f3cd-3e02-97f0-5b395827453f
      public static final UUID emptyVersion;
 -    public static final ImmutableSet<String> systemKeyspaceNames = ImmutableSet.of(Keyspace.SYSTEM_KS);
 -    public static final ImmutableSet<String> replicatedSystemKeyspaceNames = ImmutableSet.of(Tracing.TRACE_KS, Auth.AUTH_KS);
 +
++    private static final ImmutableSet<String> replicatedSystemKeyspaceNames = ImmutableSet.of(TraceKeyspace.NAME,
++                                                                                              AuthKeyspace.NAME,
++                                                                                              SystemDistributedKeyspace.NAME);
+ 
      static
      {
          try
@@@ -299,8 -291,15 +307,15 @@@
          return keyspaces.get(keyspaceName);
      }
  
+     private Set<String> getNonSystemKeyspacesSet()
+     {
 -        return Sets.difference(keyspaces.keySet(), systemKeyspaceNames);
++        return Sets.difference(keyspaces.keySet(), Collections.singleton(SystemKeyspace.NAME));
+     }
+ 
      /**
-      * @return collection of the non-system keyspaces
+      * @return collection of the non-system keyspaces (note that this count as system only the
+      * non replicated keyspaces, so keyspace like system_traces which are replicated are actually
+      * returned. See getUserKeyspace() below if you don't want those)
       */
      public List<String> getNonSystemKeyspaces()
      {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6f5dddd9/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6f5dddd9/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6f5dddd9/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------


[03/16] cassandra git commit: Fix nodetool status logic to detect if their is more than 1 user keyspace

Posted by sl...@apache.org.
Fix nodetool status logic to detect if their is more than 1 user keyspace

explanation

patch by krummas & slebresne; reviewed by krummas & slebresne for CASSANDRA-10176


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

Branch: refs/heads/cassandra-3.0
Commit: f4b21f5ffa962de19901d1841f89f529ccc335b3
Parents: 393c6ec
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 3 21:04:17 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:23:54 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/config/Schema.java     | 22 ++++++++++++++++++--
 .../locator/AbstractReplicationStrategy.java    |  5 +++++
 .../locator/NetworkTopologyStrategy.java        |  6 ++++++
 .../cassandra/service/StorageService.java       | 22 ++++++++++++--------
 5 files changed, 45 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9f51291..9a9c4f7 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.14
+ * Fix incorrect warning in 'nodetool status' (CASSANDRA-10176)
  * Properly release sstable ref when doing offline scrub (CASSANDRA-10697)
  * Improve nodetool status performance for large cluster (CASSANDRA-7238)
  * Make it clear what DTCS timestamp_resolution is used for (CASSANDRA-11041)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/config/Schema.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Schema.java b/src/java/org/apache/cassandra/config/Schema.java
index fada670..739c8ca 100644
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@ -28,12 +28,14 @@ import com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.auth.Auth;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.ConcurrentBiMap;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
@@ -67,6 +69,7 @@ public class Schema
     // 59adb24e-f3cd-3e02-97f0-5b395827453f
     public static final UUID emptyVersion;
     public static final ImmutableSet<String> systemKeyspaceNames = ImmutableSet.of(Keyspace.SYSTEM_KS);
+    public static final ImmutableSet<String> replicatedSystemKeyspaceNames = ImmutableSet.of(Tracing.TRACE_KS, Auth.AUTH_KS);
 
     static
     {
@@ -288,12 +291,27 @@ public class Schema
         return keyspaces.get(keyspaceName);
     }
 
+    private Set<String> getNonSystemKeyspacesSet()
+    {
+        return Sets.difference(keyspaces.keySet(), systemKeyspaceNames);
+    }
+
     /**
-     * @return collection of the non-system keyspaces
+     * @return collection of the non-system keyspaces (note that this count as system only the
+     * non replicated keyspaces, so keyspace like system_traces which are replicated are actually
+     * returned. See getUserKeyspace() below if you don't want those)
      */
     public List<String> getNonSystemKeyspaces()
     {
-        return ImmutableList.copyOf(Sets.difference(keyspaces.keySet(), systemKeyspaceNames));
+        return ImmutableList.copyOf(getNonSystemKeyspacesSet());
+    }
+
+    /**
+     * @return collection of the user defined keyspaces
+     */
+    public List<String> getUserKeyspaces()
+    {
+        return ImmutableList.copyOf(Sets.difference(getNonSystemKeyspacesSet(), replicatedSystemKeyspaceNames));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
index b6599f1..2178e17 100644
--- a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
@@ -303,6 +303,11 @@ public abstract class AbstractReplicationStrategy
         return strategyClass;
     }
 
+    public boolean hasSameSettings(AbstractReplicationStrategy other)
+    {
+        return getClass().equals(other.getClass()) && getReplicationFactor() == other.getReplicationFactor();
+    }
+
     protected void validateReplicationFactor(String rf) throws ConfigurationException
     {
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
index 5e5ecb7..a792aa8 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -199,4 +199,10 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
         // We explicitely allow all options
         return null;
     }
+
+    @Override
+    public boolean hasSameSettings(AbstractReplicationStrategy other)
+    {
+        return super.hasSameSettings(other) && ((NetworkTopologyStrategy) other).datacenters.equals(datacenters);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 02f6cf4..98e2251 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -4032,18 +4032,22 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
         else
         {
-            List<String> nonSystemKeyspaces = Schema.instance.getNonSystemKeyspaces();
+            List<String> userKeyspaces = Schema.instance.getUserKeyspaces();
 
-            //system_traces is a non-system keyspace however it needs to be counted as one for this process
-            int specialTableCount = 0;
-            if (nonSystemKeyspaces.contains("system_traces"))
+            if (userKeyspaces.size() > 0)
             {
-                specialTableCount += 1;
+                keyspace = userKeyspaces.iterator().next();
+                AbstractReplicationStrategy replicationStrategy = Schema.instance.getKeyspaceInstance(keyspace).getReplicationStrategy();
+                for (String keyspaceName : userKeyspaces)
+                {
+                    if (!Schema.instance.getKeyspaceInstance(keyspaceName).getReplicationStrategy().hasSameSettings(replicationStrategy))
+                        throw new IllegalStateException("Non-system keyspaces don't have the same replication settings, effective ownership information is meaningless");
+                }
+            }
+            else
+            {
+                keyspace = "system_traces";
             }
-            if (nonSystemKeyspaces.size() > specialTableCount)
-                throw new IllegalStateException("Non-system keyspaces don't have the same replication settings, effective ownership information is meaningless");
-
-            keyspace = "system_traces";
 
             Keyspace keyspaceInstance = Schema.instance.getKeyspaceInstance(keyspace);
             if (keyspaceInstance == null)


[05/16] cassandra git commit: Merge commit 'f4b21f5' into cassandra-2.2

Posted by sl...@apache.org.
Merge commit 'f4b21f5' 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/6f5dddd9
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6f5dddd9
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6f5dddd9

Branch: refs/heads/cassandra-2.2
Commit: 6f5dddd966fc16a20f2af8add74b70a9bb11b559
Parents: f3b5ff8 f4b21f5
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 17 11:25:04 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:25:31 2016 +0100

----------------------------------------------------------------------
 .../org/apache/cassandra/config/Schema.java     | 27 ++++++++++++++++++--
 .../locator/AbstractReplicationStrategy.java    |  5 ++++
 .../locator/NetworkTopologyStrategy.java        |  6 +++++
 .../cassandra/service/StorageService.java       | 22 +++++++++-------
 4 files changed, 49 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6f5dddd9/src/java/org/apache/cassandra/config/Schema.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/Schema.java
index 110029e,739c8ca..2cd7611
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@@ -26,19 -28,16 +27,22 @@@ import com.google.common.collect.Sets
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
 -import org.apache.cassandra.auth.Auth;
++import org.apache.cassandra.auth.AuthKeyspace;
 +import org.apache.cassandra.cql3.functions.Functions;
 +import org.apache.cassandra.cql3.functions.UDAggregate;
 +import org.apache.cassandra.cql3.functions.UDFunction;
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.Keyspace;
 +import org.apache.cassandra.db.commitlog.CommitLog;
 +import org.apache.cassandra.db.compaction.CompactionManager;
 +import org.apache.cassandra.db.marshal.UserType;
  import org.apache.cassandra.db.index.SecondaryIndex;
 -import org.apache.cassandra.db.index.SecondaryIndexManager;
  import org.apache.cassandra.io.sstable.Descriptor;
 +import org.apache.cassandra.schema.LegacySchemaTables;
++import org.apache.cassandra.repair.SystemDistributedKeyspace;
  import org.apache.cassandra.service.MigrationManager;
 -import org.apache.cassandra.tracing.Tracing;
++import org.apache.cassandra.tracing.TraceKeyspace;
  import org.apache.cassandra.utils.ConcurrentBiMap;
 -import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.Pair;
  import org.cliffc.high_scale_lib.NonBlockingHashMap;
  
@@@ -69,7 -68,9 +73,11 @@@ public class Schem
  
      // 59adb24e-f3cd-3e02-97f0-5b395827453f
      public static final UUID emptyVersion;
 -    public static final ImmutableSet<String> systemKeyspaceNames = ImmutableSet.of(Keyspace.SYSTEM_KS);
 -    public static final ImmutableSet<String> replicatedSystemKeyspaceNames = ImmutableSet.of(Tracing.TRACE_KS, Auth.AUTH_KS);
 +
++    private static final ImmutableSet<String> replicatedSystemKeyspaceNames = ImmutableSet.of(TraceKeyspace.NAME,
++                                                                                              AuthKeyspace.NAME,
++                                                                                              SystemDistributedKeyspace.NAME);
+ 
      static
      {
          try
@@@ -299,8 -291,15 +307,15 @@@
          return keyspaces.get(keyspaceName);
      }
  
+     private Set<String> getNonSystemKeyspacesSet()
+     {
 -        return Sets.difference(keyspaces.keySet(), systemKeyspaceNames);
++        return Sets.difference(keyspaces.keySet(), Collections.singleton(SystemKeyspace.NAME));
+     }
+ 
      /**
-      * @return collection of the non-system keyspaces
+      * @return collection of the non-system keyspaces (note that this count as system only the
+      * non replicated keyspaces, so keyspace like system_traces which are replicated are actually
+      * returned. See getUserKeyspace() below if you don't want those)
       */
      public List<String> getNonSystemKeyspaces()
      {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6f5dddd9/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6f5dddd9/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6f5dddd9/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------


[13/16] cassandra git commit: Add missing CHANGES entry

Posted by sl...@apache.org.
Add missing CHANGES entry


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

Branch: refs/heads/cassandra-2.2
Commit: f7c75857da50a73bfa63f206bdd3573c167b46c4
Parents: 6f5dddd
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 17 11:31:31 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:31:31 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7c75857/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 49bc581..5c912a1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@
  * Fix paging on DISTINCT queries repeats result when first row in partition changes
    (CASSANDRA-10010)
 Merged from 2.1:
+ * Fix incorrect warning in 'nodetool status' (CASSANDRA-10176)
  * Properly release sstable ref when doing offline scrub (CASSANDRA-10697)
  * Improve nodetool status performance for large cluster (CASSANDRA-7238)
  * Gossiper#isEnabled is not thread safe (CASSANDRA-11116)


[15/16] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

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

Branch: refs/heads/cassandra-3.0
Commit: 931d2e5f3b8005e6bbb0593550399542386e849d
Parents: ce39f81 f7c7585
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 17 11:31:39 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:31:39 2016 +0100

----------------------------------------------------------------------

----------------------------------------------------------------------



[02/16] cassandra git commit: Fix nodetool status logic to detect if their is more than 1 user keyspace

Posted by sl...@apache.org.
Fix nodetool status logic to detect if their is more than 1 user keyspace

explanation

patch by krummas & slebresne; reviewed by krummas & slebresne for CASSANDRA-10176


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

Branch: refs/heads/cassandra-2.2
Commit: f4b21f5ffa962de19901d1841f89f529ccc335b3
Parents: 393c6ec
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 3 21:04:17 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:23:54 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/config/Schema.java     | 22 ++++++++++++++++++--
 .../locator/AbstractReplicationStrategy.java    |  5 +++++
 .../locator/NetworkTopologyStrategy.java        |  6 ++++++
 .../cassandra/service/StorageService.java       | 22 ++++++++++++--------
 5 files changed, 45 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9f51291..9a9c4f7 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.14
+ * Fix incorrect warning in 'nodetool status' (CASSANDRA-10176)
  * Properly release sstable ref when doing offline scrub (CASSANDRA-10697)
  * Improve nodetool status performance for large cluster (CASSANDRA-7238)
  * Make it clear what DTCS timestamp_resolution is used for (CASSANDRA-11041)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/config/Schema.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Schema.java b/src/java/org/apache/cassandra/config/Schema.java
index fada670..739c8ca 100644
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@ -28,12 +28,14 @@ import com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.auth.Auth;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.ConcurrentBiMap;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
@@ -67,6 +69,7 @@ public class Schema
     // 59adb24e-f3cd-3e02-97f0-5b395827453f
     public static final UUID emptyVersion;
     public static final ImmutableSet<String> systemKeyspaceNames = ImmutableSet.of(Keyspace.SYSTEM_KS);
+    public static final ImmutableSet<String> replicatedSystemKeyspaceNames = ImmutableSet.of(Tracing.TRACE_KS, Auth.AUTH_KS);
 
     static
     {
@@ -288,12 +291,27 @@ public class Schema
         return keyspaces.get(keyspaceName);
     }
 
+    private Set<String> getNonSystemKeyspacesSet()
+    {
+        return Sets.difference(keyspaces.keySet(), systemKeyspaceNames);
+    }
+
     /**
-     * @return collection of the non-system keyspaces
+     * @return collection of the non-system keyspaces (note that this count as system only the
+     * non replicated keyspaces, so keyspace like system_traces which are replicated are actually
+     * returned. See getUserKeyspace() below if you don't want those)
      */
     public List<String> getNonSystemKeyspaces()
     {
-        return ImmutableList.copyOf(Sets.difference(keyspaces.keySet(), systemKeyspaceNames));
+        return ImmutableList.copyOf(getNonSystemKeyspacesSet());
+    }
+
+    /**
+     * @return collection of the user defined keyspaces
+     */
+    public List<String> getUserKeyspaces()
+    {
+        return ImmutableList.copyOf(Sets.difference(getNonSystemKeyspacesSet(), replicatedSystemKeyspaceNames));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
index b6599f1..2178e17 100644
--- a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
@@ -303,6 +303,11 @@ public abstract class AbstractReplicationStrategy
         return strategyClass;
     }
 
+    public boolean hasSameSettings(AbstractReplicationStrategy other)
+    {
+        return getClass().equals(other.getClass()) && getReplicationFactor() == other.getReplicationFactor();
+    }
+
     protected void validateReplicationFactor(String rf) throws ConfigurationException
     {
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
index 5e5ecb7..a792aa8 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -199,4 +199,10 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
         // We explicitely allow all options
         return null;
     }
+
+    @Override
+    public boolean hasSameSettings(AbstractReplicationStrategy other)
+    {
+        return super.hasSameSettings(other) && ((NetworkTopologyStrategy) other).datacenters.equals(datacenters);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 02f6cf4..98e2251 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -4032,18 +4032,22 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
         else
         {
-            List<String> nonSystemKeyspaces = Schema.instance.getNonSystemKeyspaces();
+            List<String> userKeyspaces = Schema.instance.getUserKeyspaces();
 
-            //system_traces is a non-system keyspace however it needs to be counted as one for this process
-            int specialTableCount = 0;
-            if (nonSystemKeyspaces.contains("system_traces"))
+            if (userKeyspaces.size() > 0)
             {
-                specialTableCount += 1;
+                keyspace = userKeyspaces.iterator().next();
+                AbstractReplicationStrategy replicationStrategy = Schema.instance.getKeyspaceInstance(keyspace).getReplicationStrategy();
+                for (String keyspaceName : userKeyspaces)
+                {
+                    if (!Schema.instance.getKeyspaceInstance(keyspaceName).getReplicationStrategy().hasSameSettings(replicationStrategy))
+                        throw new IllegalStateException("Non-system keyspaces don't have the same replication settings, effective ownership information is meaningless");
+                }
+            }
+            else
+            {
+                keyspace = "system_traces";
             }
-            if (nonSystemKeyspaces.size() > specialTableCount)
-                throw new IllegalStateException("Non-system keyspaces don't have the same replication settings, effective ownership information is meaningless");
-
-            keyspace = "system_traces";
 
             Keyspace keyspaceInstance = Schema.instance.getKeyspaceInstance(keyspace);
             if (keyspaceInstance == null)


[04/16] cassandra git commit: Fix nodetool status logic to detect if their is more than 1 user keyspace

Posted by sl...@apache.org.
Fix nodetool status logic to detect if their is more than 1 user keyspace

explanation

patch by krummas & slebresne; reviewed by krummas & slebresne for CASSANDRA-10176


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

Branch: refs/heads/trunk
Commit: f4b21f5ffa962de19901d1841f89f529ccc335b3
Parents: 393c6ec
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 3 21:04:17 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:23:54 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/config/Schema.java     | 22 ++++++++++++++++++--
 .../locator/AbstractReplicationStrategy.java    |  5 +++++
 .../locator/NetworkTopologyStrategy.java        |  6 ++++++
 .../cassandra/service/StorageService.java       | 22 ++++++++++++--------
 5 files changed, 45 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9f51291..9a9c4f7 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.14
+ * Fix incorrect warning in 'nodetool status' (CASSANDRA-10176)
  * Properly release sstable ref when doing offline scrub (CASSANDRA-10697)
  * Improve nodetool status performance for large cluster (CASSANDRA-7238)
  * Make it clear what DTCS timestamp_resolution is used for (CASSANDRA-11041)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/config/Schema.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Schema.java b/src/java/org/apache/cassandra/config/Schema.java
index fada670..739c8ca 100644
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@ -28,12 +28,14 @@ import com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.auth.Auth;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.ConcurrentBiMap;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
@@ -67,6 +69,7 @@ public class Schema
     // 59adb24e-f3cd-3e02-97f0-5b395827453f
     public static final UUID emptyVersion;
     public static final ImmutableSet<String> systemKeyspaceNames = ImmutableSet.of(Keyspace.SYSTEM_KS);
+    public static final ImmutableSet<String> replicatedSystemKeyspaceNames = ImmutableSet.of(Tracing.TRACE_KS, Auth.AUTH_KS);
 
     static
     {
@@ -288,12 +291,27 @@ public class Schema
         return keyspaces.get(keyspaceName);
     }
 
+    private Set<String> getNonSystemKeyspacesSet()
+    {
+        return Sets.difference(keyspaces.keySet(), systemKeyspaceNames);
+    }
+
     /**
-     * @return collection of the non-system keyspaces
+     * @return collection of the non-system keyspaces (note that this count as system only the
+     * non replicated keyspaces, so keyspace like system_traces which are replicated are actually
+     * returned. See getUserKeyspace() below if you don't want those)
      */
     public List<String> getNonSystemKeyspaces()
     {
-        return ImmutableList.copyOf(Sets.difference(keyspaces.keySet(), systemKeyspaceNames));
+        return ImmutableList.copyOf(getNonSystemKeyspacesSet());
+    }
+
+    /**
+     * @return collection of the user defined keyspaces
+     */
+    public List<String> getUserKeyspaces()
+    {
+        return ImmutableList.copyOf(Sets.difference(getNonSystemKeyspacesSet(), replicatedSystemKeyspaceNames));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
index b6599f1..2178e17 100644
--- a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
@@ -303,6 +303,11 @@ public abstract class AbstractReplicationStrategy
         return strategyClass;
     }
 
+    public boolean hasSameSettings(AbstractReplicationStrategy other)
+    {
+        return getClass().equals(other.getClass()) && getReplicationFactor() == other.getReplicationFactor();
+    }
+
     protected void validateReplicationFactor(String rf) throws ConfigurationException
     {
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
index 5e5ecb7..a792aa8 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -199,4 +199,10 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
         // We explicitely allow all options
         return null;
     }
+
+    @Override
+    public boolean hasSameSettings(AbstractReplicationStrategy other)
+    {
+        return super.hasSameSettings(other) && ((NetworkTopologyStrategy) other).datacenters.equals(datacenters);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f4b21f5f/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 02f6cf4..98e2251 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -4032,18 +4032,22 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
         else
         {
-            List<String> nonSystemKeyspaces = Schema.instance.getNonSystemKeyspaces();
+            List<String> userKeyspaces = Schema.instance.getUserKeyspaces();
 
-            //system_traces is a non-system keyspace however it needs to be counted as one for this process
-            int specialTableCount = 0;
-            if (nonSystemKeyspaces.contains("system_traces"))
+            if (userKeyspaces.size() > 0)
             {
-                specialTableCount += 1;
+                keyspace = userKeyspaces.iterator().next();
+                AbstractReplicationStrategy replicationStrategy = Schema.instance.getKeyspaceInstance(keyspace).getReplicationStrategy();
+                for (String keyspaceName : userKeyspaces)
+                {
+                    if (!Schema.instance.getKeyspaceInstance(keyspaceName).getReplicationStrategy().hasSameSettings(replicationStrategy))
+                        throw new IllegalStateException("Non-system keyspaces don't have the same replication settings, effective ownership information is meaningless");
+                }
+            }
+            else
+            {
+                keyspace = "system_traces";
             }
-            if (nonSystemKeyspaces.size() > specialTableCount)
-                throw new IllegalStateException("Non-system keyspaces don't have the same replication settings, effective ownership information is meaningless");
-
-            keyspace = "system_traces";
 
             Keyspace keyspaceInstance = Schema.instance.getKeyspaceInstance(keyspace);
             if (keyspaceInstance == null)


[07/16] cassandra git commit: Merge commit 'f4b21f5' into cassandra-2.2

Posted by sl...@apache.org.
Merge commit 'f4b21f5' 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/6f5dddd9
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6f5dddd9
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6f5dddd9

Branch: refs/heads/cassandra-3.0
Commit: 6f5dddd966fc16a20f2af8add74b70a9bb11b559
Parents: f3b5ff8 f4b21f5
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 17 11:25:04 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:25:31 2016 +0100

----------------------------------------------------------------------
 .../org/apache/cassandra/config/Schema.java     | 27 ++++++++++++++++++--
 .../locator/AbstractReplicationStrategy.java    |  5 ++++
 .../locator/NetworkTopologyStrategy.java        |  6 +++++
 .../cassandra/service/StorageService.java       | 22 +++++++++-------
 4 files changed, 49 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6f5dddd9/src/java/org/apache/cassandra/config/Schema.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/Schema.java
index 110029e,739c8ca..2cd7611
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@@ -26,19 -28,16 +27,22 @@@ import com.google.common.collect.Sets
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
 -import org.apache.cassandra.auth.Auth;
++import org.apache.cassandra.auth.AuthKeyspace;
 +import org.apache.cassandra.cql3.functions.Functions;
 +import org.apache.cassandra.cql3.functions.UDAggregate;
 +import org.apache.cassandra.cql3.functions.UDFunction;
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.Keyspace;
 +import org.apache.cassandra.db.commitlog.CommitLog;
 +import org.apache.cassandra.db.compaction.CompactionManager;
 +import org.apache.cassandra.db.marshal.UserType;
  import org.apache.cassandra.db.index.SecondaryIndex;
 -import org.apache.cassandra.db.index.SecondaryIndexManager;
  import org.apache.cassandra.io.sstable.Descriptor;
 +import org.apache.cassandra.schema.LegacySchemaTables;
++import org.apache.cassandra.repair.SystemDistributedKeyspace;
  import org.apache.cassandra.service.MigrationManager;
 -import org.apache.cassandra.tracing.Tracing;
++import org.apache.cassandra.tracing.TraceKeyspace;
  import org.apache.cassandra.utils.ConcurrentBiMap;
 -import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.Pair;
  import org.cliffc.high_scale_lib.NonBlockingHashMap;
  
@@@ -69,7 -68,9 +73,11 @@@ public class Schem
  
      // 59adb24e-f3cd-3e02-97f0-5b395827453f
      public static final UUID emptyVersion;
 -    public static final ImmutableSet<String> systemKeyspaceNames = ImmutableSet.of(Keyspace.SYSTEM_KS);
 -    public static final ImmutableSet<String> replicatedSystemKeyspaceNames = ImmutableSet.of(Tracing.TRACE_KS, Auth.AUTH_KS);
 +
++    private static final ImmutableSet<String> replicatedSystemKeyspaceNames = ImmutableSet.of(TraceKeyspace.NAME,
++                                                                                              AuthKeyspace.NAME,
++                                                                                              SystemDistributedKeyspace.NAME);
+ 
      static
      {
          try
@@@ -299,8 -291,15 +307,15 @@@
          return keyspaces.get(keyspaceName);
      }
  
+     private Set<String> getNonSystemKeyspacesSet()
+     {
 -        return Sets.difference(keyspaces.keySet(), systemKeyspaceNames);
++        return Sets.difference(keyspaces.keySet(), Collections.singleton(SystemKeyspace.NAME));
+     }
+ 
      /**
-      * @return collection of the non-system keyspaces
+      * @return collection of the non-system keyspaces (note that this count as system only the
+      * non replicated keyspaces, so keyspace like system_traces which are replicated are actually
+      * returned. See getUserKeyspace() below if you don't want those)
       */
      public List<String> getNonSystemKeyspaces()
      {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6f5dddd9/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6f5dddd9/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6f5dddd9/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------


[16/16] cassandra git commit: Merge branch 'cassandra-3.0' into trunk

Posted by sl...@apache.org.
Merge branch 'cassandra-3.0' into trunk


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

Branch: refs/heads/trunk
Commit: 09fd433cb5b2455cba9ff1e9267e521119f26ecd
Parents: b53260d 931d2e5
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 17 11:31:53 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:31:53 2016 +0100

----------------------------------------------------------------------

----------------------------------------------------------------------



[14/16] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

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

Branch: refs/heads/trunk
Commit: 931d2e5f3b8005e6bbb0593550399542386e849d
Parents: ce39f81 f7c7585
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 17 11:31:39 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:31:39 2016 +0100

----------------------------------------------------------------------

----------------------------------------------------------------------



[08/16] cassandra git commit: Merge commit '6f5dddd' into cassandra-3.0

Posted by sl...@apache.org.
Merge commit '6f5dddd' 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/ce39f81d
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ce39f81d
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ce39f81d

Branch: refs/heads/cassandra-3.0
Commit: ce39f81d26489ba292447a18a66057980b0c3f53
Parents: 9cfbc31 6f5dddd
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 17 11:25:53 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:27:27 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/config/Schema.java     | 28 ++++++++++++++++++--
 .../locator/AbstractReplicationStrategy.java    |  5 ++++
 .../locator/NetworkTopologyStrategy.java        |  6 +++++
 .../cassandra/service/StorageService.java       | 22 ++++++++-------
 5 files changed, 51 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ce39f81d/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 52fc3ab,49bc581..32c165b
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -22,6 -9,6 +22,7 @@@ Merged from 2.2
   * Fix paging on DISTINCT queries repeats result when first row in partition changes
     (CASSANDRA-10010)
  Merged from 2.1:
++ * Fix incorrect warning in 'nodetool status' (CASSANDRA-10176)
   * Properly release sstable ref when doing offline scrub (CASSANDRA-10697)
   * Improve nodetool status performance for large cluster (CASSANDRA-7238)
   * Gossiper#isEnabled is not thread safe (CASSANDRA-11116)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ce39f81d/src/java/org/apache/cassandra/config/Schema.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/config/Schema.java
index 7ffa89e,2cd7611..3fd9f11
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@@ -27,18 -27,21 +27,21 @@@ import com.google.common.collect.Sets
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
+ import org.apache.cassandra.auth.AuthKeyspace;
 -import org.apache.cassandra.cql3.functions.Functions;
 -import org.apache.cassandra.cql3.functions.UDAggregate;
 -import org.apache.cassandra.cql3.functions.UDFunction;
 -import org.apache.cassandra.db.*;
 +import org.apache.cassandra.cql3.functions.*;
 +import org.apache.cassandra.db.ColumnFamilyStore;
  import org.apache.cassandra.db.Keyspace;
 +import org.apache.cassandra.db.SystemKeyspace;
  import org.apache.cassandra.db.commitlog.CommitLog;
  import org.apache.cassandra.db.compaction.CompactionManager;
 +import org.apache.cassandra.db.marshal.AbstractType;
  import org.apache.cassandra.db.marshal.UserType;
 -import org.apache.cassandra.db.index.SecondaryIndex;
 +import org.apache.cassandra.index.Index;
  import org.apache.cassandra.io.sstable.Descriptor;
 -import org.apache.cassandra.schema.LegacySchemaTables;
+ import org.apache.cassandra.repair.SystemDistributedKeyspace;
 +import org.apache.cassandra.schema.*;
  import org.apache.cassandra.service.MigrationManager;
+ import org.apache.cassandra.tracing.TraceKeyspace;
  import org.apache.cassandra.utils.ConcurrentBiMap;
  import org.apache.cassandra.utils.Pair;
  import org.cliffc.high_scale_lib.NonBlockingHashMap;
@@@ -49,9 -52,6 +52,14 @@@ public class Schem
  
      public static final Schema instance = new Schema();
  
 +    /* system keyspace names (the ones with LocalStrategy replication strategy) */
 +    public static final Set<String> SYSTEM_KEYSPACE_NAMES = ImmutableSet.of(SystemKeyspace.NAME, SchemaKeyspace.NAME);
 +
++    /* replicate system keyspace names (the ones with a "true" replication strategy) */
++    public static final Set<String> REPLICATED_SYSTEM_KEYSPACE_NAMES = ImmutableSet.of(TraceKeyspace.NAME,
++                                                                                       AuthKeyspace.NAME,
++                                                                                       SystemDistributedKeyspace.NAME);
++
      /**
       * longest permissible KS or CF name.  Our main concern is that filename not be more than 255 characters;
       * the filename will contain both the KS and CF names. Since non-schema-name components only take up
@@@ -74,6 -74,10 +82,7 @@@
      // 59adb24e-f3cd-3e02-97f0-5b395827453f
      public static final UUID emptyVersion;
  
 -    private static final ImmutableSet<String> replicatedSystemKeyspaceNames = ImmutableSet.of(TraceKeyspace.NAME,
 -                                                                                              AuthKeyspace.NAME,
 -                                                                                              SystemDistributedKeyspace.NAME);
+ 
      static
      {
          try
@@@ -323,12 -307,27 +332,27 @@@
          return keyspaces.get(keyspaceName);
      }
  
+     private Set<String> getNonSystemKeyspacesSet()
+     {
 -        return Sets.difference(keyspaces.keySet(), Collections.singleton(SystemKeyspace.NAME));
++        return Sets.difference(keyspaces.keySet(), SYSTEM_KEYSPACE_NAMES);
+     }
+ 
      /**
-      * @return collection of the non-system keyspaces
+      * @return collection of the non-system keyspaces (note that this count as system only the
+      * non replicated keyspaces, so keyspace like system_traces which are replicated are actually
+      * returned. See getUserKeyspace() below if you don't want those)
       */
      public List<String> getNonSystemKeyspaces()
      {
-         return ImmutableList.copyOf(Sets.difference(keyspaces.keySet(), SYSTEM_KEYSPACE_NAMES));
+         return ImmutableList.copyOf(getNonSystemKeyspacesSet());
+     }
+ 
+     /**
+      * @return collection of the user defined keyspaces
+      */
+     public List<String> getUserKeyspaces()
+     {
 -        return ImmutableList.copyOf(Sets.difference(getNonSystemKeyspacesSet(), replicatedSystemKeyspaceNames));
++        return ImmutableList.copyOf(Sets.difference(getNonSystemKeyspacesSet(), REPLICATED_SYSTEM_KEYSPACE_NAMES));
      }
  
      /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ce39f81d/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 7cca516,0e716e8..185c988
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -4157,18 -4045,22 +4157,22 @@@ public class StorageService extends Not
          }
          else
          {
-             List<String> nonSystemKeyspaces = Schema.instance.getNonSystemKeyspaces();
+             List<String> userKeyspaces = Schema.instance.getUserKeyspaces();
  
-             //system_traces is a non-system keyspace however it needs to be counted as one for this process
-             int specialTableCount = 0;
-             if (nonSystemKeyspaces.contains("system_traces"))
+             if (userKeyspaces.size() > 0)
              {
-                 specialTableCount += 1;
 -                keyspace = userKeyspaces.iterator().next();
++                keyspace = userKeyspaces.get(0);
+                 AbstractReplicationStrategy replicationStrategy = Schema.instance.getKeyspaceInstance(keyspace).getReplicationStrategy();
+                 for (String keyspaceName : userKeyspaces)
+                 {
+                     if (!Schema.instance.getKeyspaceInstance(keyspaceName).getReplicationStrategy().hasSameSettings(replicationStrategy))
+                         throw new IllegalStateException("Non-system keyspaces don't have the same replication settings, effective ownership information is meaningless");
+                 }
+             }
+             else
+             {
+                 keyspace = "system_traces";
              }
-             if (nonSystemKeyspaces.size() > specialTableCount)
-                 throw new IllegalStateException("Non-system keyspaces don't have the same replication settings, effective ownership information is meaningless");
- 
-             keyspace = "system_traces";
  
              Keyspace keyspaceInstance = Schema.instance.getKeyspaceInstance(keyspace);
              if (keyspaceInstance == null)


[10/16] cassandra git commit: Merge branch 'cassandra-3.0' into trunk

Posted by sl...@apache.org.
Merge branch 'cassandra-3.0' into trunk


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

Branch: refs/heads/trunk
Commit: b53260d30691dbaf25098db6f3d2837ba1a0f9f9
Parents: 48815d4 ce39f81
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 17 11:30:58 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:30:58 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/config/Schema.java     | 28 ++++++++++++++++++--
 .../locator/AbstractReplicationStrategy.java    |  5 ++++
 .../locator/NetworkTopologyStrategy.java        |  6 +++++
 .../cassandra/service/StorageService.java       | 22 ++++++++-------
 5 files changed, 51 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b53260d3/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
index f0342d4,7c8d95e..756b689
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@@ -223,4 -192,16 +223,10 @@@ public class NetworkTopologyStrategy ex
              validateReplicationFactor(e.getValue());
          }
      }
+ 
 -    public Collection<String> recognizedOptions()
 -    {
 -        // We explicitely allow all options
 -        return null;
 -    }
 -
+     @Override
+     public boolean hasSameSettings(AbstractReplicationStrategy other)
+     {
+         return super.hasSameSettings(other) && ((NetworkTopologyStrategy) other).datacenters.equals(datacenters);
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b53260d3/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------


[12/16] cassandra git commit: Add missing CHANGES entry

Posted by sl...@apache.org.
Add missing CHANGES entry


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

Branch: refs/heads/trunk
Commit: f7c75857da50a73bfa63f206bdd3573c167b46c4
Parents: 6f5dddd
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Feb 17 11:31:31 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Feb 17 11:31:31 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7c75857/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 49bc581..5c912a1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@
  * Fix paging on DISTINCT queries repeats result when first row in partition changes
    (CASSANDRA-10010)
 Merged from 2.1:
+ * Fix incorrect warning in 'nodetool status' (CASSANDRA-10176)
  * Properly release sstable ref when doing offline scrub (CASSANDRA-10697)
  * Improve nodetool status performance for large cluster (CASSANDRA-7238)
  * Gossiper#isEnabled is not thread safe (CASSANDRA-11116)