You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by yu...@apache.org on 2016/05/05 22:40:53 UTC

cassandra git commit: Remove DatabaseDescriptor dependency from FileUtils

Repository: cassandra
Updated Branches:
  refs/heads/trunk 89eb3e58b -> 1dd33eca1


Remove DatabaseDescriptor dependency from FileUtils

patch by yukim; reviewed by snazy for CASSANDRA-11578


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

Branch: refs/heads/trunk
Commit: 1dd33eca1b72e6cf1261471f88e7c3990be3f097
Parents: 89eb3e5
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue Apr 12 11:26:14 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu May 5 17:33:10 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../org/apache/cassandra/io/FSErrorHandler.java |  30 ++++++
 .../org/apache/cassandra/io/util/FileUtils.java |  75 +++-----------
 .../cassandra/service/CassandraDaemon.java      |   2 +
 .../service/DefaultFSErrorHandler.java          | 101 +++++++++++++++++++
 .../apache/cassandra/db/DirectoriesTest.java    |   2 +
 6 files changed, 150 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1dd33eca/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 45d9fb1..ba64a19 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.8
+ * Remove DatabaseDescriptor dependency from FileUtils (CASSANDRA-11578)
  * Faster streaming (CASSANDRA-9766)
 
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1dd33eca/src/java/org/apache/cassandra/io/FSErrorHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/FSErrorHandler.java b/src/java/org/apache/cassandra/io/FSErrorHandler.java
new file mode 100644
index 0000000..081ec0b
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/FSErrorHandler.java
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+
+/**
+ * Interface for handling file system errors.
+ */
+public interface FSErrorHandler
+{
+    void handleCorruptSSTable(CorruptSSTableException e);
+    void handleFSError(FSError e);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1dd33eca/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java
index 6b58e85..d522c27 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -28,27 +28,25 @@ import java.text.DecimalFormat;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
-
-import sun.nio.ch.DirectBuffer;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import sun.nio.ch.DirectBuffer;
 
 import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.BlacklistedDirectories;
-import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.FSError;
+import org.apache.cassandra.io.FSErrorHandler;
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
 import static org.apache.cassandra.utils.Throwables.maybeFail;
 import static org.apache.cassandra.utils.Throwables.merge;
 
-public class FileUtils
+public final class FileUtils
 {
     public static final Charset CHARSET = StandardCharsets.UTF_8;
 
@@ -60,6 +58,7 @@ public class FileUtils
 
     private static final DecimalFormat df = new DecimalFormat("#.##");
     private static final boolean canCleanDirectBuffers;
+    private static final AtomicReference<Optional<FSErrorHandler>> fsErrorHandler = new AtomicReference<>(Optional.empty());
 
     static
     {
@@ -463,63 +462,12 @@ public class FileUtils
 
     public static void handleCorruptSSTable(CorruptSSTableException e)
     {
-        if (!StorageService.instance.isDaemonSetupCompleted())
-            handleStartupFSError(e);
-
-        JVMStabilityInspector.inspectThrowable(e);
-        switch (DatabaseDescriptor.getDiskFailurePolicy())
-        {
-            case stop_paranoid:
-                StorageService.instance.stopTransports();
-                break;
-        }
-    }
-    
-    public static void handleFSError(FSError e)
-    {
-        if (!StorageService.instance.isDaemonSetupCompleted())
-            handleStartupFSError(e);
-
-        JVMStabilityInspector.inspectThrowable(e);
-        switch (DatabaseDescriptor.getDiskFailurePolicy())
-        {
-            case stop_paranoid:
-            case stop:
-                StorageService.instance.stopTransports();
-                break;
-            case best_effort:
-                // for both read and write errors mark the path as unwritable.
-                BlacklistedDirectories.maybeMarkUnwritable(e.path);
-                if (e instanceof FSReadError)
-                {
-                    File directory = BlacklistedDirectories.maybeMarkUnreadable(e.path);
-                    if (directory != null)
-                        Keyspace.removeUnreadableSSTables(directory);
-                }
-                break;
-            case ignore:
-                // already logged, so left nothing to do
-                break;
-            default:
-                throw new IllegalStateException();
-        }
+        fsErrorHandler.get().ifPresent(handler -> handler.handleCorruptSSTable(e));
     }
 
-    private static void handleStartupFSError(Throwable t)
+    public static void handleFSError(FSError e)
     {
-        switch (DatabaseDescriptor.getDiskFailurePolicy())
-        {
-            case stop_paranoid:
-            case stop:
-            case die:
-                logger.error("Exiting forcefully due to file system exception on startup, disk failure policy \"{}\"",
-                             DatabaseDescriptor.getDiskFailurePolicy(),
-                             t);
-                JVMStabilityInspector.killCurrentJVM(t, true);
-                break;
-            default:
-                break;
-        }
+        fsErrorHandler.get().ifPresent(handler -> handler.handleFSError(e));
     }
 
     /**
@@ -634,4 +582,9 @@ public class FileUtils
             throw new RuntimeException(ex);
         }
     }
+
+    public static void setFSErrorHandler(FSErrorHandler handler)
+    {
+        fsErrorHandler.getAndSet(Optional.ofNullable(handler));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1dd33eca/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index b1fd457..2edf0b6 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -159,6 +159,8 @@ public class CassandraDaemon
      */
     protected void setup()
     {
+        FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
+
         // Delete any failed snapshot deletions on Windows - see CASSANDRA-9658
         if (FBUtilities.isWindows())
             WindowsFailedSnapshotTracker.deleteOldSnapshots();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1dd33eca/src/java/org/apache/cassandra/service/DefaultFSErrorHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/DefaultFSErrorHandler.java b/src/java/org/apache/cassandra/service/DefaultFSErrorHandler.java
new file mode 100644
index 0000000..c653683
--- /dev/null
+++ b/src/java/org/apache/cassandra/service/DefaultFSErrorHandler.java
@@ -0,0 +1,101 @@
+/*
+ * 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.service;
+
+import java.io.File;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.BlacklistedDirectories;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.io.FSError;
+import org.apache.cassandra.io.FSErrorHandler;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+public class DefaultFSErrorHandler implements FSErrorHandler
+{
+    private static final Logger logger = LoggerFactory.getLogger(DefaultFSErrorHandler.class);
+
+    @Override
+    public void handleCorruptSSTable(CorruptSSTableException e)
+    {
+        if (!StorageService.instance.isDaemonSetupCompleted())
+            handleStartupFSError(e);
+
+        JVMStabilityInspector.inspectThrowable(e);
+        switch (DatabaseDescriptor.getDiskFailurePolicy())
+        {
+            case stop_paranoid:
+                StorageService.instance.stopTransports();
+                break;
+        }
+    }
+
+    @Override
+    public void handleFSError(FSError e)
+    {
+        if (!StorageService.instance.isDaemonSetupCompleted())
+            handleStartupFSError(e);
+
+        JVMStabilityInspector.inspectThrowable(e);
+        switch (DatabaseDescriptor.getDiskFailurePolicy())
+        {
+            case stop_paranoid:
+            case stop:
+                StorageService.instance.stopTransports();
+                break;
+            case best_effort:
+                // for both read and write errors mark the path as unwritable.
+                BlacklistedDirectories.maybeMarkUnwritable(e.path);
+                if (e instanceof FSReadError)
+                {
+                    File directory = BlacklistedDirectories.maybeMarkUnreadable(e.path);
+                    if (directory != null)
+                        Keyspace.removeUnreadableSSTables(directory);
+                }
+                break;
+            case ignore:
+                // already logged, so left nothing to do
+                break;
+            default:
+                throw new IllegalStateException();
+        }
+    }
+
+    private static void handleStartupFSError(Throwable t)
+    {
+        switch (DatabaseDescriptor.getDiskFailurePolicy())
+        {
+            case stop_paranoid:
+            case stop:
+            case die:
+                logger.error("Exiting forcefully due to file system exception on startup, disk failure policy \"{}\"",
+                             DatabaseDescriptor.getDiskFailurePolicy(),
+                             t);
+                JVMStabilityInspector.killCurrentJVM(t, true);
+                break;
+            default:
+                break;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1dd33eca/test/unit/org/apache/cassandra/db/DirectoriesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/DirectoriesTest.java b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
index f864bbc..f8d01a8 100644
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@ -43,6 +43,7 @@ import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.schema.IndexMetadata;
+import org.apache.cassandra.service.DefaultFSErrorHandler;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 
@@ -65,6 +66,7 @@ public class DirectoriesTest
     @BeforeClass
     public static void beforeClass() throws IOException
     {
+        FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
         for (String table : TABLES)
         {
             UUID tableID = CFMetaData.generateLegacyCfId(KS, table);