You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jm...@apache.org on 2015/09/03 19:53:18 UTC

cassandra git commit: Retry snapshot deletion after compaction and gc on Windows

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.2 e1fc0e067 -> 9b8e18702


Retry snapshot deletion after compaction and gc on Windows

Patch by jmckenzie; reviewed by stefania for CASSANDRA-10222


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

Branch: refs/heads/cassandra-2.2
Commit: 9b8e1870294d4b3b277e74e7bef13e52b6b3d57e
Parents: e1fc0e0
Author: Joshua McKenzie <jm...@apache.org>
Authored: Thu Sep 3 13:49:43 2015 -0400
Committer: Joshua McKenzie <jm...@apache.org>
Committed: Thu Sep 3 13:49:43 2015 -0400

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../org/apache/cassandra/db/Directories.java    |   7 +-
 .../db/WindowsFailedSnapshotTracker.java        |   2 +-
 .../db/compaction/CompactionManager.java        |   4 +
 .../io/sstable/SSTableDeletingTask.java         |   3 +
 .../io/sstable/SnapshotDeletingTask.java        |  83 ++++++++++++++
 .../db/commitlog/SnapshotDeletingTest.java      | 109 +++++++++++++++++++
 .../db/compaction/CompactionsTest.java          |   3 +-
 8 files changed, 204 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8e1870/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index fcb9ed7..20e1297 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.2
+ * Retry snapshot deletion after compaction and gc on Windows (CASSANDRA-10222)
  * Fix failure to start with space in directory path on Windows (CASSANDRA-10239)
  * Fix repair hang when snapshot failed (CASSANDRA-10057)
  * Fall back to 1/4 commitlog volume for commitlog_total_space on small disks

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8e1870/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 8b61c68..0011baf 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -736,14 +736,9 @@ public class Directories
                 catch (FSWriteError e)
                 {
                     if (FBUtilities.isWindows())
-                    {
-                        logger.warn("Failed to delete snapshot directory [{}]. Folder will be deleted on JVM shutdown or next node restart on crash. You can safely attempt to delete this folder but it will fail so long as readers are open on the files.", snapshotDir);
-                        WindowsFailedSnapshotTracker.handleFailedSnapshot(snapshotDir);
-                    }
+                        SnapshotDeletingTask.addFailedSnapshot(snapshotDir);
                     else
-                    {
                         throw e;
-                    }
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8e1870/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java b/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java
index ce89823..9e6bb47 100644
--- a/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java
+++ b/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java
@@ -101,7 +101,7 @@ public class WindowsFailedSnapshotTracker
 
     public static synchronized void handleFailedSnapshot(File dir)
     {
-        assert(_failedSnapshotFile != null);
+        assert _failedSnapshotFile != null : "_failedSnapshotFile not initialized within WindowsFailedSnapshotTracker";
         FileUtils.deleteRecursiveOnExit(dir);
         _failedSnapshotFile.println(dir.toString());
         _failedSnapshotFile.flush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8e1870/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 5bac084..5e1b31c 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -1441,6 +1441,10 @@ public class CompactionManager implements CompactionManagerMBean
                     DebuggableThreadPoolExecutor.handleOrLog(t);
                 }
             }
+
+            // Snapshots cannot be deleted on Windows while segments of the root element are mapped in NTFS. Compactions
+            // unmap those segments which could free up a snapshot for successful deletion.
+            SnapshotDeletingTask.rescheduleFailedTasks();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8e1870/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java b/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
index ec6439c..34d0347 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
@@ -106,6 +106,9 @@ public class SSTableDeletingTask implements Runnable
             failedTasks.remove(task);
             task.schedule();
         }
+
+        // On Windows, snapshots cannot be deleted so long as a segment of the root element is memory-mapped in NTFS.
+        SnapshotDeletingTask.rescheduleFailedTasks();
     }
 
     /** for tests */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8e1870/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java b/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java
new file mode 100644
index 0000000..7ab1f11
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java
@@ -0,0 +1,83 @@
+/*
+ * 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.io.sstable;
+
+import java.io.File;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArraySet;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.db.WindowsFailedSnapshotTracker;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.util.FileUtils;
+
+public class SnapshotDeletingTask implements Runnable
+{
+    private static final Logger logger = LoggerFactory.getLogger(SnapshotDeletingTask.class);
+
+    public final File path;
+    private static final Set<SnapshotDeletingTask> failedTasks = new CopyOnWriteArraySet<>();
+
+    public static void addFailedSnapshot(File path)
+    {
+        logger.warn("Failed to delete snapshot [{}]. Will retry after further sstable deletions. Folder will be deleted on JVM shutdown or next node restart on crash.", path);
+        WindowsFailedSnapshotTracker.handleFailedSnapshot(path);
+        failedTasks.add(new SnapshotDeletingTask(path));
+    }
+
+    private SnapshotDeletingTask(File path)
+    {
+        this.path = path;
+    }
+
+    public void run()
+    {
+        try
+        {
+            FileUtils.deleteRecursive(path);
+            logger.info("Successfully deleted snapshot {}.", path);
+        }
+        catch (FSWriteError e)
+        {
+            failedTasks.add(this);
+        }
+    }
+
+    /**
+     * Retry all failed deletions.
+     */
+    public static void rescheduleFailedTasks()
+    {
+        for (SnapshotDeletingTask task : failedTasks)
+        {
+            failedTasks.remove(task);
+            ScheduledExecutors.nonPeriodicTasks.submit(task);
+        }
+    }
+
+    @VisibleForTesting
+    public static int pendingDeletionCount()
+    {
+        return failedTasks.size();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8e1870/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java b/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java
new file mode 100644
index 0000000..99cf72d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java
@@ -0,0 +1,109 @@
+/*
+ * 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.db.commitlog;
+
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.WindowsFailedSnapshotTracker;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.SnapshotDeletingTask;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.service.GCInspector;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+public class SnapshotDeletingTest
+{
+    private static final String KEYSPACE1 = "Keyspace1";
+    private static final String CF_STANDARD1 = "CF_STANDARD1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
+    @Test
+    public void testSnapshotDeletionFailure() throws Exception
+    {
+        Assume.assumeTrue(FBUtilities.isWindows());
+
+        GCInspector.register();
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARD1);
+        store.clearUnsafe();
+
+        // Needed to init the output file where we print failed snapshots. This is called on node startup.
+        WindowsFailedSnapshotTracker.deleteOldSnapshots();
+
+        populate(10000);
+        store.snapshot("snapshot1");
+
+        // Confirm snapshot deletion fails. Sleep for a bit just to make sure the SnapshotDeletingTask has
+        // time to run and fail.
+        Thread.sleep(500);
+        store.clearSnapshot("snapshot1");
+        assertEquals(1, SnapshotDeletingTask.pendingDeletionCount());
+
+        // Compact the cf and confirm that the executor's after hook calls rescheduleDeletion
+        populate(20000);
+        store.forceBlockingFlush();
+        store.forceMajorCompaction();
+
+        long start = System.currentTimeMillis();
+        while (System.currentTimeMillis() - start < 1000 && SnapshotDeletingTask.pendingDeletionCount() > 0)
+        {
+            Thread.yield();
+        }
+
+        assertEquals(0, SnapshotDeletingTask.pendingDeletionCount());
+    }
+
+    private long populate(int rowCount)
+    {
+        long timestamp = System.currentTimeMillis();
+        for (int i = 0; i <= rowCount; i++)
+        {
+            DecoratedKey key = Util.dk(Integer.toString(i));
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
+            for (int j = 0; j < 10; j++)
+                rm.add(CF_STANDARD1,  Util.cellname(Integer.toString(j)),
+                       ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                       timestamp,
+                       0);
+            rm.applyUnsafe();
+        }
+        return timestamp;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9b8e1870/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
index 9dee746..8ff3022 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
@@ -125,7 +125,8 @@ public class CompactionsTest
         return store;
     }
 
-    private long populate(String ks, String cf, int startRowKey, int endRowKey, int ttl) {
+    private long populate(String ks, String cf, int startRowKey, int endRowKey, int ttl)
+    {
         long timestamp = System.currentTimeMillis();
         for (int i = startRowKey; i <= endRowKey; i++)
         {