You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ad...@apache.org on 2021/05/07 17:00:44 UTC

[cassandra] branch cassandra-4.0 updated: Fix DROP COMPACT STORAGE for counters

This is an automated email from the ASF dual-hosted git repository.

adelapena pushed a commit to branch cassandra-4.0
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/cassandra-4.0 by this push:
     new fb76baa  Fix DROP COMPACT STORAGE for counters
fb76baa is described below

commit fb76baa60898db82831df44796bd224d30e3236d
Author: Andrés de la Peña <a....@gmail.com>
AuthorDate: Fri May 7 17:50:13 2021 +0100

    Fix DROP COMPACT STORAGE for counters
    
    patch by Andrés de la Peña; reviewed by Benjamin Lerer and Ekaterina Dimitrova for CASSANDRA-16653
---
 CHANGES.txt                                        |  1 +
 .../statements/schema/AlterTableStatement.java     |  8 ++-
 .../cassandra/distributed/test/CountersTest.java   | 78 ++++++++++++++++++++++
 3 files changed, 86 insertions(+), 1 deletion(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 0edae1b..a424dfc 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0-rc2
+ * Fix DROP COMPACT STORAGE for counters (CASSANDRA-16653)
  * Add back validation for AlterTableStatements (CASSANDRA-16643)
  * Fix cqlsh encoding error with unicode in multi-line statement (CASSANDRA-16539)
  * Fix race in fat client removal (CASSANDRA-16238)
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
index 0492172..5e3bfa2 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
@@ -72,6 +72,8 @@ import static java.lang.String.join;
 import static com.google.common.collect.Iterables.isEmpty;
 import static com.google.common.collect.Iterables.transform;
 
+import static org.apache.cassandra.schema.TableMetadata.Flag;
+
 public abstract class AlterTableStatement extends AlterSchemaStatement
 {
     protected final String tableName;
@@ -434,7 +436,11 @@ public abstract class AlterTableStatement extends AlterSchemaStatement
 
             validateCanDropCompactStorage();
 
-            return keyspace.withSwapped(keyspace.tables.withSwapped(table.withSwapped(ImmutableSet.of(TableMetadata.Flag.COMPOUND))));
+            Set<Flag> flags = table.isCounter()
+                            ? ImmutableSet.of(Flag.COMPOUND, Flag.COUNTER)
+                            : ImmutableSet.of(Flag.COMPOUND);
+
+            return keyspace.withSwapped(keyspace.tables.withSwapped(table.withSwapped(flags)));
         }
 
         /**
diff --git a/test/distributed/org/apache/cassandra/distributed/test/CountersTest.java b/test/distributed/org/apache/cassandra/distributed/test/CountersTest.java
new file mode 100644
index 0000000..4e1db58
--- /dev/null
+++ b/test/distributed/org/apache/cassandra/distributed/test/CountersTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.distributed.test;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.ICoordinator;
+
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+public class CountersTest extends TestBaseImpl
+{
+    @Test
+    public void testUpdateCounter() throws Throwable
+    {
+        testUpdateCounter(false);
+    }
+
+    @Test
+    public void testUpdateCounterWithDroppedCompactStorage() throws Throwable
+    {
+        testUpdateCounter(true);
+    }
+
+    private static void testUpdateCounter(boolean droppedCompactStorage) throws Throwable
+    {
+        try (Cluster cluster = Cluster.build(2).withConfig(c -> c.with(GOSSIP, NATIVE_PROTOCOL)).start())
+        {
+            cluster.schemaChange("CREATE KEYSPACE k WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}");
+
+            String createTable = "CREATE TABLE k.t ( k int, c int, total counter, PRIMARY KEY (k, c))";
+            if (droppedCompactStorage)
+            {
+                cluster.schemaChange(createTable + " WITH COMPACT STORAGE");
+                cluster.schemaChange("ALTER TABLE k.t DROP COMPACT STORAGE");
+            }
+            else
+            {
+                cluster.schemaChange(createTable);
+            }
+
+            ConsistencyLevel cl = ConsistencyLevel.ONE;
+            String select = "SELECT total FROM k.t WHERE k = 1 AND c = ?";
+
+            for (int i = 1; i <= cluster.size(); i++)
+            {
+                ICoordinator coordinator = cluster.coordinator(i);
+
+                coordinator.execute("UPDATE k.t SET total = total + 1 WHERE k = 1 AND c = ?", cl, i);
+                assertRows(coordinator.execute(select, cl, i), row(1L));
+
+                coordinator.execute("UPDATE k.t SET total = total - 4 WHERE k = 1 AND c = ?", cl, i);
+                assertRows(coordinator.execute(select, cl, i), row(-3L));
+            }
+        }
+    }
+}

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org