You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2014/06/27 11:30:12 UTC

[1/2] git commit: Improve schema merge performance

Repository: cassandra
Updated Branches:
  refs/heads/trunk beea5d68c -> 39bdd3b08


Improve schema merge performance

patch by Aleksey Yeschenko; reviewed by Sylvain Lebresne for
CASSANDRA-7444


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

Branch: refs/heads/trunk
Commit: 9da949bbeed9d924f98e7cc977e864162caba2c9
Parents: 8aa2b79
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Fri Jun 27 02:19:24 2014 -0700
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Fri Jun 27 02:19:24 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/config/KSMetaData.java |  2 +-
 .../org/apache/cassandra/db/DefsTables.java     | 18 +++++++++-----
 .../org/apache/cassandra/db/SystemKeyspace.java | 25 ++++++++++++++------
 .../cassandra/service/MigrationManager.java     |  2 +-
 5 files changed, 33 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9da949bb/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 95921ff..64f2bee 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.0
+ * Improve schema merge performance (CASSANDRA-7444)
 Merged from 2.0:
  * Fix CC#collectTimeOrderedData() tombstone optimisations (CASSANDRA-7394)
  * Support DISTINCT for static columns and fix behaviour when DISTINC is

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9da949bb/src/java/org/apache/cassandra/config/KSMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/KSMetaData.java b/src/java/org/apache/cassandra/config/KSMetaData.java
index 7700394..8c99191 100644
--- a/src/java/org/apache/cassandra/config/KSMetaData.java
+++ b/src/java/org/apache/cassandra/config/KSMetaData.java
@@ -224,7 +224,7 @@ public final class KSMetaData
 
     public KSMetaData reloadAttributes()
     {
-        Row ksDefRow = SystemKeyspace.readSchemaRow(name);
+        Row ksDefRow = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_KEYSPACES_CF, name);
 
         if (ksDefRow.cf == null)
             throw new RuntimeException(String.format("%s not found in the schema definitions keyspaceName (%s).", name, SystemKeyspace.SCHEMA_KEYSPACES_CF));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9da949bb/src/java/org/apache/cassandra/db/DefsTables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DefsTables.java b/src/java/org/apache/cassandra/db/DefsTables.java
index ede3ebd..bd1987e 100644
--- a/src/java/org/apache/cassandra/db/DefsTables.java
+++ b/src/java/org/apache/cassandra/db/DefsTables.java
@@ -40,6 +40,7 @@ import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * SCHEMA_{KEYSPACES, COLUMNFAMILIES, COLUMNS}_CF are used to store Keyspace/ColumnFamily attributes to make schema
@@ -167,10 +168,15 @@ public class DefsTables
 
     public static synchronized void mergeSchemaInternal(Collection<Mutation> mutations, boolean doFlush) throws ConfigurationException, IOException
     {
+        // compare before/after schemas of the affected keyspaces only
+        Set<String> keyspaces = new HashSet<>(mutations.size());
+        for (Mutation mutation : mutations)
+            keyspaces.add(ByteBufferUtil.string(mutation.key()));
+
         // current state of the schema
-        Map<DecoratedKey, ColumnFamily> oldKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_CF);
-        Map<DecoratedKey, ColumnFamily> oldColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF);
-        Map<DecoratedKey, ColumnFamily> oldTypes = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_USER_TYPES_CF);
+        Map<DecoratedKey, ColumnFamily> oldKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_CF, keyspaces);
+        Map<DecoratedKey, ColumnFamily> oldColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF, keyspaces);
+        Map<DecoratedKey, ColumnFamily> oldTypes = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_USER_TYPES_CF, keyspaces);
 
         for (Mutation mutation : mutations)
             mutation.apply();
@@ -179,9 +185,9 @@ public class DefsTables
             flushSchemaCFs();
 
         // with new data applied
-        Map<DecoratedKey, ColumnFamily> newKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_CF);
-        Map<DecoratedKey, ColumnFamily> newColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF);
-        Map<DecoratedKey, ColumnFamily> newTypes = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_USER_TYPES_CF);
+        Map<DecoratedKey, ColumnFamily> newKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_CF, keyspaces);
+        Map<DecoratedKey, ColumnFamily> newColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF, keyspaces);
+        Map<DecoratedKey, ColumnFamily> newTypes = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_USER_TYPES_CF, keyspaces);
 
         Set<String> keyspacesToDrop = mergeKeyspaces(oldKeyspaces, newKeyspaces);
         mergeColumnFamilies(oldColumnFamilies, newColumnFamilies);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9da949bb/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 659bc69..3c647b6 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -769,12 +769,16 @@ public class SystemKeyspace
         }
     }
 
-    public static Map<DecoratedKey, ColumnFamily> getSchema(String cfName)
+    public static Map<DecoratedKey, ColumnFamily> getSchema(String schemaCfName, Set<String> keyspaces)
     {
-        Map<DecoratedKey, ColumnFamily> schema = new HashMap<DecoratedKey, ColumnFamily>();
+        Map<DecoratedKey, ColumnFamily> schema = new HashMap<>();
 
-        for (Row schemaEntity : SystemKeyspace.serializedSchema(cfName))
-            schema.put(schemaEntity.key, schemaEntity.cf);
+        for (String keyspace : keyspaces)
+        {
+            Row schemaEntity = readSchemaRow(schemaCfName, keyspace);
+            if (schemaEntity.cf != null)
+                schema.put(schemaEntity.key, schemaEntity.cf);
+        }
 
         return schema;
     }
@@ -784,12 +788,19 @@ public class SystemKeyspace
         return AsciiType.instance.fromString(ksName);
     }
 
-    public static Row readSchemaRow(String ksName)
+    /**
+     * Fetches a subset of schema (table data, columns metadata or triggers) for the keyspace.
+     *
+     * @param schemaCfName the schema table to get the data from (schema_keyspaces, schema_columnfamilies, schema_columns or schema_triggers)
+     * @param ksName the keyspace of the tables we are interested in
+     * @return a Row containing the schema data of a particular type for the keyspace
+     */
+    public static Row readSchemaRow(String schemaCfName, String ksName)
     {
         DecoratedKey key = StorageService.getPartitioner().decorateKey(getSchemaKSKey(ksName));
 
-        ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(SCHEMA_KEYSPACES_CF);
-        ColumnFamily result = schemaCFS.getColumnFamily(QueryFilter.getIdentityFilter(key, SCHEMA_KEYSPACES_CF, System.currentTimeMillis()));
+        ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(schemaCfName);
+        ColumnFamily result = schemaCFS.getColumnFamily(QueryFilter.getIdentityFilter(key, schemaCfName, System.currentTimeMillis()));
 
         return new Row(key, result);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9da949bb/src/java/org/apache/cassandra/service/MigrationManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationManager.java b/src/java/org/apache/cassandra/service/MigrationManager.java
index a6408ec..6d28f62 100644
--- a/src/java/org/apache/cassandra/service/MigrationManager.java
+++ b/src/java/org/apache/cassandra/service/MigrationManager.java
@@ -338,7 +338,7 @@ public class MigrationManager
     // Include the serialized keyspace for when a target node missed the CREATE KEYSPACE migration (see #5631).
     private static Mutation addSerializedKeyspace(Mutation migration, String ksName)
     {
-        migration.add(SystemKeyspace.readSchemaRow(ksName).cf);
+        migration.add(SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_KEYSPACES_CF, ksName).cf);
         return migration;
     }
 


[2/2] git commit: Merge branch 'cassandra-2.1' into trunk

Posted by al...@apache.org.
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 39bdd3b081b60ffe8a41fa8bf776a598987c69cb
Parents: beea5d6 9da949b
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Fri Jun 27 02:29:09 2014 -0700
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Fri Jun 27 02:29:09 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/config/KSMetaData.java |  2 +-
 .../org/apache/cassandra/db/DefsTables.java     | 18 +++++++++-----
 .../org/apache/cassandra/db/SystemKeyspace.java | 25 ++++++++++++++------
 .../cassandra/service/MigrationManager.java     |  2 +-
 5 files changed, 33 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/39bdd3b0/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index dad329f,64f2bee..4308f71
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,17 -1,5 +1,18 @@@
 +3.0
 + * Permit configurable timestamps with cassandra-stress (CASSANDRA-7416)
 + * Move sstable RandomAccessReader to nio2, which allows using the
 +   FILE_SHARE_DELETE flag on Windows (CASSANDRA-4050)
 + * Remove CQL2 (CASSANDRA-5918)
 + * Add Thrift get_multi_slice call (CASSANDRA-6757)
 + * Optimize fetching multiple cells by name (CASSANDRA-6933)
 + * Allow compilation in java 8 (CASSANDRA-7208)
 + * Make incremental repair default (CASSANDRA-7250)
 + * Enable code coverage thru JaCoCo (CASSANDRA-7226)
 + * Switch external naming of 'column families' to 'tables' (CASSANDRA-4369) 
 +
 +
  2.1.0
+  * Improve schema merge performance (CASSANDRA-7444)
  Merged from 2.0:
   * Fix CC#collectTimeOrderedData() tombstone optimisations (CASSANDRA-7394)
   * Support DISTINCT for static columns and fix behaviour when DISTINC is

http://git-wip-us.apache.org/repos/asf/cassandra/blob/39bdd3b0/src/java/org/apache/cassandra/service/MigrationManager.java
----------------------------------------------------------------------