You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2022/05/12 21:03:17 UTC

[GitHub] [cassandra] pauloricardomg commented on a diff in pull request #1615: CASSANDRA-17619: Remove expired snapshots of dropped tables after restart

pauloricardomg commented on code in PR #1615:
URL: https://github.com/apache/cassandra/pull/1615#discussion_r871763069


##########
src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java:
##########
@@ -116,8 +108,15 @@ public synchronized void addSnapshot(TableSnapshot snapshot)
     @VisibleForTesting
     protected synchronized void loadSnapshots()
     {
-        logger.debug("Loading snapshots");
-        snapshotLoader.get().forEach(this::addSnapshot);
+        SnapshotLoader loader = new SnapshotLoader(DatabaseDescriptor.getAllDataFileLocations());

Review Comment:
   > Dont you think we might have loader as a property of SnapshotManager ? Why do we need to create an instance every time?
   
   Since this is a bug fix I would prefer not to make loader a property of SnapshotManager because it's out of the scope of this task.
   
   The SnapshotLoader class was originally designed to be used only once, like a builder. In order to make the class reusable I would need to change the design and update tests so I would prefer to leave for post 4.1.
   
   After 4.1 is out I will create a new PR with a major refactoring moving all snapshot logic to SnapshotManager, based on this branch implementation:
   * https://github.com/pauloricardomg/cassandra/tree/CASSANDRA-16843-final
   
   > We are also creating it in StorageService.getSnapshotDetails. Can not we somehow make it by one instance?
   
   same as above, prefer to not change this for now but will do later as part of the refactoring.
   
   Is this ok for now? I can add a comment with a small TODO if you'd like.



##########
src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java:
##########
@@ -116,8 +108,15 @@ public synchronized void addSnapshot(TableSnapshot snapshot)
     @VisibleForTesting
     protected synchronized void loadSnapshots()
     {
-        logger.debug("Loading snapshots");
-        snapshotLoader.get().forEach(this::addSnapshot);
+        SnapshotLoader loader = new SnapshotLoader(DatabaseDescriptor.getAllDataFileLocations());
+        addSnapshots(loader.loadSnapshots());
+    }
+
+    @VisibleForTesting
+    protected synchronized void addSnapshots(Collection<TableSnapshot> snapshots)
+    {
+        logger.debug("Adding snapshots: {}", snapshots.stream().map(s -> s.getId()).collect(Collectors.toList()));

Review Comment:
   Updated. New output:
   ```
   DEBUG [main] 2022-05-12 17:08:30,773 SnapshotManager.java:119 - Adding snapshots: ks:tbl:c7483545-103a-493e-b66e-6d96e60d3878:expired, ks:tbl:984bd8f1-35fd-44d7-b69b-52422614e25f:non-expired, ks:tbl:deff9907-495b-4497-8275-ffdfdfc3ad7b:non-expiring.
   ```



##########
src/java/org/apache/cassandra/service/snapshot/SnapshotManager.java:
##########
@@ -116,8 +108,15 @@ public synchronized void addSnapshot(TableSnapshot snapshot)
     @VisibleForTesting
     protected synchronized void loadSnapshots()
     {
-        logger.debug("Loading snapshots");
-        snapshotLoader.get().forEach(this::addSnapshot);
+        SnapshotLoader loader = new SnapshotLoader(DatabaseDescriptor.getAllDataFileLocations());

Review Comment:
   > btw, I just went through that SnapshotLoader class as such and I would change one of the constructors to this:
   
   updated 👍 



##########
test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java:
##########
@@ -222,6 +224,71 @@ public void testListSnapshotOfDroppedTable()
         instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains("tag1");
     }
 
+    @Test
+    public void testTTLSnapshotOfDroppedTable()
+    {
+        IInvokableInstance instance = cluster.get(1);
+
+        cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS default WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");

Review Comment:
   > why rf 2?
   
   copied over from previous tests - fixed all tests to use RF=1 on 007958f8746799f4ca89a51b82bdd9d20eae1494.
   
   > Also, can no you use that distributed keyspace which is created already?
   
   This whole class uses this format and fixing would be out of the scope of this ticket. Is it fine if we refactor this later?



##########
test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java:
##########
@@ -222,6 +224,71 @@ public void testListSnapshotOfDroppedTable()
         instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains("tag1");
     }
 
+    @Test
+    public void testTTLSnapshotOfDroppedTable()
+    {
+        IInvokableInstance instance = cluster.get(1);
+
+        cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS default WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");
+        cluster.schemaChange("CREATE TABLE default.tbl (key int, value text, PRIMARY KEY (key))");
+
+        populate(cluster);
+
+        instance.nodetoolResult("snapshot",
+                                "-t", "tag1",
+                                "-kt", "default.tbl",
+                                "--ttl", String.format("%ds", FIVE_SECONDS)).asserts().success();
+
+        // Check snapshot is listed when table is not dropped
+        instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains("tag1");
+
+        // Drop Table
+        cluster.schemaChange("DROP TABLE default.tbl;");
+
+        // Check snapshot is listed after table is dropped
+        instance.nodetoolResult("listsnapshots").asserts().success().stdoutContains("tag1");
+
+        // Check snapshot is removed after at most 10s
+        await().timeout(2 * FIVE_SECONDS, SECONDS)
+               .pollInterval(1, SECONDS)
+               .until(() -> !instance.nodetoolResult("listsnapshots").getStdout().contains("tag1"));
+    }
+
+    @Test
+    public void testTTLSnapshotOfDroppedTableAfterRestart()
+    {
+        int TWENTY_SECONDS = 20; // longer TTL to allow snapshot to survive node restart
+        IInvokableInstance instance = cluster.get(1);
+
+        cluster.schemaChange("CREATE KEYSPACE IF NOT EXISTS default WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 2};");

Review Comment:
   fixed to use rf=1 on 007958f8746799f4ca89a51b82bdd9d20eae1494



##########
test/distributed/org/apache/cassandra/distributed/test/SnapshotsTest.java:
##########
@@ -35,7 +35,10 @@
 import org.apache.cassandra.distributed.api.NodeToolResult;
 import org.apache.cassandra.distributed.shared.WithProperties;
 
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.db.ColumnFamilyStore.SNAPSHOT_DROP_PREFIX;
 import static org.apache.cassandra.distributed.shared.ClusterUtils.stopUnchecked;
+import static org.awaitility.Awaitility.await;
 
 public class SnapshotsTest extends TestBaseImpl

Review Comment:
   good catch - added new test on 834229664d6c2b3b6a044e7fd2571eca48df6a74



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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