You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2015/12/15 10:29:27 UTC

cassandra git commit: Allow CREATE TABLE WITH ID

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 9b30d6572 -> cee35e42d


Allow CREATE TABLE WITH ID

Patch by Branimir Lambov; reviewed by marcuse for CASSANDRA-9179


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

Branch: refs/heads/cassandra-2.1
Commit: cee35e42d05f125bf274b9cefb7653c21ca4b676
Parents: 9b30d65
Author: Branimir Lambov <br...@datastax.com>
Authored: Tue Dec 8 11:59:09 2015 +0200
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Tue Dec 15 09:58:42 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../org/apache/cassandra/config/CFMetaData.java |   6 +-
 .../cql3/statements/AlterTableStatement.java    |   2 +
 .../cassandra/cql3/statements/CFPropDefs.java   |  23 +++-
 .../cql3/statements/CreateTableStatement.java   |   9 +-
 .../operations/DropRecreateAndRestoreTest.java  | 105 +++++++++++++++++++
 6 files changed, 136 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/cee35e42/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 091ac52..7f1d66b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.13
+ * Allow CREATE TABLE WITH ID (CASSANDRA-9179)
  * Make Stress compiles within eclipse (CASSANDRA-10807)
  * Cassandra Daemon should print JVM arguments (CASSANDRA-10764)
  * Allow cancellation of index summary redistribution (CASSANDRA-8805)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cee35e42/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 2939f09..bce5c72 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -470,12 +470,12 @@ public final class CFMetaData
      */
     public CFMetaData(String keyspace, String name, ColumnFamilyType type, CellNameType comp)
     {
-        this(keyspace, name, type, comp, UUIDGen.getTimeUUID());
+        this(keyspace, name, type, comp, null);
     }
 
-    private CFMetaData(String keyspace, String name, ColumnFamilyType type, CellNameType comp, UUID id)
+    public CFMetaData(String keyspace, String name, ColumnFamilyType type, CellNameType comp, UUID id)
     {
-        cfId = id;
+        cfId = id != null ? id : UUIDGen.getTimeUUID();
         ksName = keyspace;
         cfName = name;
         ksAndCFName = Pair.create(keyspace, name);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cee35e42/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index 44bc2bc..42b7f7f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -87,6 +87,8 @@ public class AlterTableStatement extends SchemaAlteringStatement
             columnName = rawColumnName.prepare(cfm);
             def = cfm.getColumnDefinition(columnName);
         }
+        if (cfProps.getId() != null)
+            throw new ConfigurationException("Cannot alter table id.");
 
         switch (oType)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cee35e42/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
index 343c0c0..17edd6d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
@@ -17,10 +17,7 @@
  */
 package org.apache.cassandra.cql3.statements;
 
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 
 import org.apache.cassandra.cache.CachingOptions;
 import org.apache.cassandra.config.CFMetaData;
@@ -49,6 +46,8 @@ public class CFPropDefs extends PropertyDefinitions
     public static final String KW_COMPACTION = "compaction";
     public static final String KW_COMPRESSION = "compression";
 
+    public static final String KW_ID = "id";
+
     public static final String COMPACTION_STRATEGY_CLASS_KEY = "class";
 
     public static final Set<String> keywords = new HashSet<>();
@@ -69,6 +68,7 @@ public class CFPropDefs extends PropertyDefinitions
         keywords.add(KW_COMPACTION);
         keywords.add(KW_COMPRESSION);
         keywords.add(KW_MEMTABLE_FLUSH_PERIOD);
+        keywords.add(KW_ID);
 
         obsoleteKeywords.add("index_interval");
         obsoleteKeywords.add("replicate_on_write");
@@ -86,6 +86,15 @@ public class CFPropDefs extends PropertyDefinitions
 
         validate(keywords, obsoleteKeywords);
 
+        try
+        {
+            getId();
+        }
+        catch (IllegalArgumentException e)
+        {
+            throw new ConfigurationException("Invalid table id", e);
+        }
+
         Map<String, String> compactionOptions = getCompactionOptions();
         if (!compactionOptions.isEmpty())
         {
@@ -170,6 +179,12 @@ public class CFPropDefs extends PropertyDefinitions
         return getInt(KW_DEFAULT_TIME_TO_LIVE, 0);
     }
 
+    public UUID getId() throws SyntaxException
+    {
+        String id = getSimple(KW_ID);
+        return id != null ? UUID.fromString(id) : null;
+    }
+
     public void applyToCFMetadata(CFMetaData cfm) throws ConfigurationException, SyntaxException
     {
         if (hasProperty(KW_COMMENT))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cee35e42/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
index 5c4e9e2..2c4b23b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
@@ -64,13 +64,15 @@ public class CreateTableStatement extends SchemaAlteringStatement
     private final Set<ColumnIdentifier> staticColumns;
     private final CFPropDefs properties;
     private final boolean ifNotExists;
+    private final UUID id;
 
-    public CreateTableStatement(CFName name, CFPropDefs properties, boolean ifNotExists, Set<ColumnIdentifier> staticColumns)
+    public CreateTableStatement(CFName name, CFPropDefs properties, boolean ifNotExists, Set<ColumnIdentifier> staticColumns, UUID id)
     {
         super(name);
         this.properties = properties;
         this.ifNotExists = ifNotExists;
         this.staticColumns = staticColumns;
+        this.id = id;
 
         try
         {
@@ -146,7 +148,8 @@ public class CreateTableStatement extends SchemaAlteringStatement
         newCFMD = new CFMetaData(keyspace(),
                                  columnFamily(),
                                  ColumnFamilyType.Standard,
-                                 comparator);
+                                 comparator,
+                                 id);
         applyPropertiesTo(newCFMD);
         return newCFMD;
     }
@@ -204,7 +207,7 @@ public class CreateTableStatement extends SchemaAlteringStatement
 
             properties.validate();
 
-            CreateTableStatement stmt = new CreateTableStatement(cfName, properties, ifNotExists, staticColumns);
+            CreateTableStatement stmt = new CreateTableStatement(cfName, properties, ifNotExists, staticColumns, properties.getId());
 
             boolean hasCounters = false;
             Map<ByteBuffer, CollectionType> definedMultiCellCollections = null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cee35e42/test/unit/org/apache/cassandra/cql3/validation/operations/DropRecreateAndRestoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DropRecreateAndRestoreTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DropRecreateAndRestoreTest.java
new file mode 100644
index 0000000..385ed3d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DropRecreateAndRestoreTest.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.validation.operations;
+
+import java.io.File;
+import java.util.List;
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.exceptions.AlreadyExistsException;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.io.util.FileUtils;
+
+public class DropRecreateAndRestoreTest extends CQLTester
+{
+    @Test
+    public void testCreateWithIdRestore() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY(a, b))");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
+
+
+        long time = System.currentTimeMillis();
+        UUID id = currentTableMetadata().cfId;
+        assertRows(execute("SELECT * FROM %s"), row(0, 0, 0), row(0, 1, 1));
+        Thread.sleep(5);
+
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 0, 2);
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 1, 3);
+        assertRows(execute("SELECT * FROM %s"), row(1, 0, 2), row(1, 1, 3), row(0, 0, 0), row(0, 1, 1));
+
+        // Drop will flush and clean segments. Hard-link them so that they can be restored later.
+        List<String> segments = CommitLog.instance.getActiveSegmentNames();
+        File logPath = new File(DatabaseDescriptor.getCommitLogLocation());
+        for (String segment: segments)
+            FileUtils.createHardLink(new File(logPath, segment), new File(logPath, segment + ".save"));
+
+        execute("DROP TABLE %s");
+
+        assertInvalidThrow(InvalidRequestException.class, "SELECT * FROM %s");
+
+        execute(String.format("CREATE TABLE %%s (a int, b int, c int, PRIMARY KEY(a, b)) WITH ID = %s", id));
+
+        // Restore saved segments
+        for (String segment: segments)
+            FileUtils.renameWithConfirm(new File(logPath, segment + ".save"), new File(logPath, segment));
+        try
+        {
+            // Restore to point in time.
+            CommitLog.instance.archiver.restorePointInTime = time;
+            CommitLog.instance.resetUnsafe();
+            CommitLog.instance.recover();
+        }
+        finally
+        {
+            CommitLog.instance.archiver.restorePointInTime = Long.MAX_VALUE;
+        }
+
+        assertRows(execute("SELECT * FROM %s"), row(0, 0, 0), row(0, 1, 1));
+    }
+
+    @Test(expected = AlreadyExistsException.class)
+    public void testCreateWithIdDuplicate() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY(a, b))");
+        UUID id = currentTableMetadata().cfId;
+        execute(String.format("CREATE TABLE %%s (a int, b int, c int, PRIMARY KEY(a, b)) WITH ID = %s", id));
+    }
+
+    @Test(expected = ConfigurationException.class)
+    public void testCreateWithIdInvalid() throws Throwable
+    {
+        createTableMayThrow(String.format("CREATE TABLE %%s (a int, b int, c int, PRIMARY KEY(a, b)) WITH ID = %s", 55));
+    }
+
+    @Test(expected = ConfigurationException.class)
+    public void testAlterWithId() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY(a, b))");
+        UUID id = currentTableMetadata().cfId;
+        execute(String.format("ALTER TABLE %%s WITH ID = %s", id));
+    }
+}