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:38 UTC

[1/4] cassandra git commit: Retry snapshot deletion after compaction and gc on Windows

Repository: cassandra
Updated Branches:
  refs/heads/trunk 829b9b78c -> 9068a31f9


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/trunk
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++)
         {


[4/4] cassandra git commit: Merge branch 'cassandra-3.0' into trunk

Posted by jm...@apache.org.
Merge branch 'cassandra-3.0' into trunk


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

Branch: refs/heads/trunk
Commit: 9068a31f9bd9f8761c9527daed280a1847405af1
Parents: 829b9b7 8ebd590
Author: Joshua McKenzie <jm...@apache.org>
Authored: Thu Sep 3 13:53:00 2015 -0400
Committer: Joshua McKenzie <jm...@apache.org>
Committed: Thu Sep 3 13:53:00 2015 -0400

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


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9068a31f/CHANGES.txt
----------------------------------------------------------------------


[3/4] cassandra git commit: 3.0 patch for CASSANDRA-10222

Posted by jm...@apache.org.
3.0 patch 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/8ebd5904
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8ebd5904
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8ebd5904

Branch: refs/heads/trunk
Commit: 8ebd5904734d102e580c95dd2cdeba22de332991
Parents: f2923d4
Author: Joshua McKenzie <jm...@apache.org>
Authored: Thu Sep 3 13:52:29 2015 -0400
Committer: Joshua McKenzie <jm...@apache.org>
Committed: Thu Sep 3 13:52:29 2015 -0400

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


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8ebd5904/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c1f6992..fccaf11 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -15,6 +15,7 @@
  * Validate gc_grace_seconds for batchlog writes and MVs (CASSANDRA-9917)
  * Fix sstablerepairedset (CASSANDRA-10132)
 Merged from 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/8ebd5904/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 90d2085..c17b1fd 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -800,14 +800,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/8ebd5904/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/8ebd5904/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 0890341..07678b5 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -54,6 +54,7 @@ import org.apache.cassandra.index.SecondaryIndexBuilder;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
 import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.SnapshotDeletingTask;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
@@ -1421,6 +1422,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/8ebd5904/src/java/org/apache/cassandra/db/lifecycle/TransactionLog.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/TransactionLog.java b/src/java/org/apache/cassandra/db/lifecycle/TransactionLog.java
index 8f83f2c..69de370 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/TransactionLog.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/TransactionLog.java
@@ -49,6 +49,7 @@ import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SnapshotDeletingTask;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.big.BigFormat;
 import org.apache.cassandra.io.util.FileUtils;
@@ -927,6 +928,9 @@ public class TransactionLog extends Transactional.AbstractTransactional implemen
         Runnable task;
         while ( null != (task = failedDeletions.poll()))
             ScheduledExecutors.nonPeriodicTasks.submit(task);
+
+        // On Windows, snapshots cannot be deleted so long as a segment of the root element is memory-mapped in NTFS.
+        SnapshotDeletingTask.rescheduleFailedTasks();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8ebd5904/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/8ebd5904/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..37f1731
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java
@@ -0,0 +1,106 @@
+/*
+ * 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.CFMetaData;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowUpdateBuilder;
+import org.apache.cassandra.db.WindowsFailedSnapshotTracker;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.SnapshotDeletingTask;
+import org.apache.cassandra.schema.KeyspaceParams;
+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 Exception
+    {
+        GCInspector.register();
+        // Needed to init the output file where we print failed snapshots. This is called on node startup.
+        WindowsFailedSnapshotTracker.deleteOldSnapshots();
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
+    @Test
+    public void testCompactionHook() throws Exception
+    {
+        Assume.assumeTrue(FBUtilities.isWindows());
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARD1);
+        store.clearUnsafe();
+
+        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 void populate(int rowCount) {
+        long timestamp = System.currentTimeMillis();
+        CFMetaData cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1).metadata;
+        for (int i = 0; i <= rowCount; i++)
+        {
+            DecoratedKey key = Util.dk(Integer.toString(i));
+            for (int j = 0; j < 10; j++)
+            {
+                new RowUpdateBuilder(cfm, timestamp, 0, key.getKey())
+                    .clustering(Integer.toString(j))
+                    .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+                    .build()
+                    .applyUnsafe();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8ebd5904/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 091bc03..198b01b 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
@@ -115,7 +115,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();
         CFMetaData cfm = Keyspace.open(ks).getColumnFamilyStore(cf).metadata;
         for (int i = startRowKey; i <= endRowKey; i++)


[2/4] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by jm...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/trunk
Commit: f2923d4c83ee00de3818877d999fb7c9a8d92f54
Parents: e9c0e17 9b8e187
Author: Joshua McKenzie <jm...@apache.org>
Authored: Thu Sep 3 13:50:17 2015 -0400
Committer: Joshua McKenzie <jm...@apache.org>
Committed: Thu Sep 3 13:50:17 2015 -0400

----------------------------------------------------------------------

----------------------------------------------------------------------