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/26 14:57:39 UTC

[1/6] cassandra git commit: Backport CASSANDRA-11578

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 5dc741497 -> b851792c4
  refs/heads/cassandra-2.2 63efa0775 -> 4aa859e51
  refs/heads/cassandra-3.0 a3d055662 -> 764f3d3a1


Backport CASSANDRA-11578

patch by yukim; reviewed by Paulo Motta for CASSANDRA-11750


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

Branch: refs/heads/cassandra-2.1
Commit: b851792c4e3ae32b8d863d9079cca6d135f1cf23
Parents: 5dc7414
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed May 18 17:03:39 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu May 26 09:40:40 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, 152 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b851792c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f73db6e..ad9d00c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.15
+ * Backport CASSANDRA-11578 (CASSANDRA-11750)
  * Clear out parent repair session if repair coordinator dies (CASSANDRA-11824)
  * Set default streaming_socket_timeout_in_ms to 24 hours (CASSANDRA-11840)
  * Do not consider local node a valid source during replace (CASSANDRA-11848)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b851792c/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/b851792c/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 3be7c99..f69ed01 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -32,6 +32,7 @@ import java.nio.file.Path;
 import java.nio.file.StandardCopyOption;
 import java.text.DecimalFormat;
 import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicReference;
 
 import sun.nio.ch.DirectBuffer;
 
@@ -39,17 +40,14 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 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;
 
-public class FileUtils
+public final class FileUtils
 {
     private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
     private static final double KB = 1024d;
@@ -59,6 +57,7 @@ public class FileUtils
 
     private static final DecimalFormat df = new DecimalFormat("#.##");
     private static final boolean canCleanDirectBuffers;
+    private static final AtomicReference<FSErrorHandler> fsErrorHandler = new AtomicReference<>();
 
     static
     {
@@ -395,63 +394,16 @@ public class FileUtils
 
     public static void handleCorruptSSTable(CorruptSSTableException e)
     {
-        if (!StorageService.instance.isSetupCompleted())
-            handleStartupFSError(e);
-
-        JVMStabilityInspector.inspectThrowable(e);
-        switch (DatabaseDescriptor.getDiskFailurePolicy())
-        {
-            case stop_paranoid:
-                StorageService.instance.stopTransports();
-                break;
-        }
-    }
-    
-    public static void handleFSError(FSError e)
-    {
-        if (!StorageService.instance.isSetupCompleted())
-            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 handler = fsErrorHandler.get();
+        if (handler != null)
+            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 handler = fsErrorHandler.get();
+        if (handler != null)
+            handler.handleFSError(e);
     }
     /**
      * Get the size of a directory in bytes
@@ -470,4 +422,9 @@ public class FileUtils
         }
         return length;
     }
+
+    public static void setFSErrorHandler(FSErrorHandler handler)
+    {
+        fsErrorHandler.getAndSet(handler);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b851792c/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 a27fa20..5a1fb14 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -148,6 +148,8 @@ public class CassandraDaemon
      */
     protected void setup()
     {
+        FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
+
         try
         {
             logger.info("Hostname: {}", InetAddress.getLocalHost().getHostName());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b851792c/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..88a1fce
--- /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.isSetupCompleted())
+            handleStartupFSError(e);
+
+        JVMStabilityInspector.inspectThrowable(e);
+        switch (DatabaseDescriptor.getDiskFailurePolicy())
+        {
+            case stop_paranoid:
+                StorageService.instance.stopTransports();
+                break;
+        }
+    }
+
+    @Override
+    public void handleFSError(FSError e)
+    {
+        if (!StorageService.instance.isSetupCompleted())
+            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/b851792c/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 b1c51ee..4267c1f 100644
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@ -44,6 +44,7 @@ import org.apache.cassandra.db.Directories.DataDirectory;
 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.service.DefaultFSErrorHandler;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.io.FSWriteError;
 
@@ -65,6 +66,7 @@ public class DirectoriesTest
     @BeforeClass
     public static void beforeClass() throws IOException
     {
+        FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
         for (String cf : CFS)
         {
             CFM.add(new CFMetaData(KS, cf, ColumnFamilyType.Standard, null));


[4/6] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by yu...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2


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

Branch: refs/heads/cassandra-3.0
Commit: 4aa859e51a38d0856ba4ee2d0c177d83722271dd
Parents: 63efa07 b851792
Author: Yuki Morishita <yu...@apache.org>
Authored: Thu May 26 09:47:04 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu May 26 09:47:04 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4aa859e5/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 190c2fa,ad9d00c..f2276f0
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,24 -1,5 +1,25 @@@
 -2.1.15
 +2.2.7
 + * Enable client encryption in sstableloader with cli options (CASSANDRA-11708)
 + * Possible memory leak in NIODataInputStream (CASSANDRA-11867)
 + * Fix commit log replay after out-of-order flush completion (CASSANDRA-9669)
 + * Add seconds to cqlsh tracing session duration (CASSANDRA-11753)
 + * Prohibit Reverse Counter type as part of the PK (CASSANDRA-9395)
 + * cqlsh: correctly handle non-ascii chars in error messages (CASSANDRA-11626)
 + * Exit JVM if JMX server fails to startup (CASSANDRA-11540)
 + * Produce a heap dump when exiting on OOM (CASSANDRA-9861)
 + * Avoid read repairing purgeable tombstones on range slices (CASSANDRA-11427)
 + * Restore ability to filter on clustering columns when using a 2i (CASSANDRA-11510)
 + * JSON datetime formatting needs timezone (CASSANDRA-11137)
 + * Fix is_dense recalculation for Thrift-updated tables (CASSANDRA-11502)
 + * Remove unnescessary file existence check during anticompaction (CASSANDRA-11660)
 + * Add missing files to debian packages (CASSANDRA-11642)
 + * Avoid calling Iterables::concat in loops during ModificationStatement::getFunctions (CASSANDRA-11621)
 + * cqlsh: COPY FROM should use regular inserts for single statement batches and
 +   report errors correctly if workers processes crash on initialization (CASSANDRA-11474)
 + * Always close cluster with connection in CqlRecordWriter (CASSANDRA-11553)
 + * Fix slice queries on ordered COMPACT tables (CASSANDRA-10988)
 +Merged from 2.1:
+  * Backport CASSANDRA-11578 (CASSANDRA-11750)
   * Clear out parent repair session if repair coordinator dies (CASSANDRA-11824)
   * Set default streaming_socket_timeout_in_ms to 24 hours (CASSANDRA-11840)
   * Do not consider local node a valid source during replace (CASSANDRA-11848)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4aa859e5/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/FileUtils.java
index ce45370,f69ed01..3124fcd
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@@ -17,14 -17,23 +17,15 @@@
   */
  package org.apache.cassandra.io.util;
  
 -import java.io.Closeable;
 -import java.io.DataInput;
 -import java.io.EOFException;
 -import java.io.File;
 -import java.io.FileNotFoundException;
 -import java.io.IOException;
 -import java.io.RandomAccessFile;
 +import java.io.*;
  import java.nio.ByteBuffer;
 -import java.nio.MappedByteBuffer;
 -import java.nio.file.AtomicMoveNotSupportedException;
 -import java.nio.file.Files;
 -import java.nio.file.Path;
 -import java.nio.file.StandardCopyOption;
 +import java.nio.channels.FileChannel;
 +import java.nio.file.*;
  import java.text.DecimalFormat;
  import java.util.Arrays;
+ import java.util.concurrent.atomic.AtomicReference;
  
 +import org.apache.cassandra.config.Config;
  import sun.nio.ch.DirectBuffer;
  
  import org.slf4j.Logger;
@@@ -38,13 -45,9 +37,12 @@@ import org.apache.cassandra.io.FSErrorH
  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
  {
      private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
      private static final double KB = 1024d;
@@@ -487,39 -423,8 +439,43 @@@
          return length;
      }
  
- 
 +    public static void copyTo(DataInput in, OutputStream out, int length) throws IOException
 +    {
 +        byte[] buffer = new byte[64 * 1024];
 +        int copiedBytes = 0;
 +
 +        while (copiedBytes + buffer.length < length)
 +        {
 +            in.readFully(buffer);
 +            out.write(buffer);
 +            copiedBytes += buffer.length;
 +        }
 +
 +        if (copiedBytes < length)
 +        {
 +            int left = length - copiedBytes;
 +            in.readFully(buffer, 0, left);
 +            out.write(buffer, 0, left);
 +        }
 +    }
 +
 +    public static boolean isSubDirectory(File parent, File child) throws IOException
 +    {
 +        parent = parent.getCanonicalFile();
 +        child = child.getCanonicalFile();
 +
 +        File toCheck = child;
 +        while (toCheck != null)
 +        {
 +            if (parent.equals(toCheck))
 +                return true;
 +            toCheck = toCheck.getParentFile();
 +        }
 +        return false;
 +    }
++
+     public static void setFSErrorHandler(FSErrorHandler handler)
+     {
+         fsErrorHandler.getAndSet(handler);
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4aa859e5/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/CassandraDaemon.java
index 7e33e9c,5a1fb14..1e9378b
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@@ -149,32 -146,81 +149,34 @@@ public class CassandraDaemo
       */
      protected void setup()
      {
+         FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
+ 
 -        try
 -        {
 -            logger.info("Hostname: {}", InetAddress.getLocalHost().getHostName());
 -        }
 -        catch (UnknownHostException e1)
 -        {
 -            logger.info("Could not resolve local host");
 -        }
 +        // Delete any failed snapshot deletions on Windows - see CASSANDRA-9658
 +        if (FBUtilities.isWindows())
 +            WindowsFailedSnapshotTracker.deleteOldSnapshots();
  
 -        long now = System.currentTimeMillis();
 -        if (now < EARLIEST_LAUNCH_DATE)
 -        {
 -            logger.error("current machine time is {}, but that is seemingly incorrect. exiting now.", new Date(now));
 -            System.exit(3);
 -        }
 +        logSystemInfo();
 +
 +        CLibrary.tryMlockall();
  
 -        // log warnings for different kinds of sub-optimal JVMs.  tldr use 64-bit Oracle >= 1.6u32
 -        if (!DatabaseDescriptor.hasLargeAddressSpace())
 -            logger.info("32bit JVM detected.  It is recommended to run Cassandra on a 64bit JVM for better performance.");
 -        String javaVersion = System.getProperty("java.version");
 -        String javaVmName = System.getProperty("java.vm.name");
 -        logger.info("JVM vendor/version: {}/{}", javaVmName, javaVersion);
 -        if (javaVmName.contains("OpenJDK"))
 +        try
          {
 -            // There is essentially no QA done on OpenJDK builds, and
 -            // clusters running OpenJDK have seen many heap and load issues.
 -            logger.warn("OpenJDK is not recommended. Please upgrade to the newest Oracle Java release");
 +            startupChecks.verify();
          }
 -        else if (!javaVmName.contains("HotSpot"))
 +        catch (StartupException e)
          {
 -            logger.warn("Non-Oracle JVM detected.  Some features, such as immediate unmap of compacted SSTables, may not work as intended");
 +            exitOrFail(e.returnCode, e.getMessage(), e.getCause());
          }
 -     /*   else
 +
 +        try
          {
 -            String[] java_version = javaVersion.split("_");
 -            String java_major = java_version[0];
 -            int java_minor;
 -            try
 -            {
 -                java_minor = (java_version.length > 1) ? Integer.parseInt(java_version[1]) : 0;
 -            }
 -            catch (NumberFormatException e)
 -            {
 -                // have only seen this with java7 so far but no doubt there are other ways to break this
 -                logger.info("Unable to parse java version {}", Arrays.toString(java_version));
 -                java_minor = 32;
 -            }
 +            SystemKeyspace.snapshotOnVersionChange();
          }
 -     */
 -        logger.info("Heap size: {}/{}", Runtime.getRuntime().totalMemory(), Runtime.getRuntime().maxMemory());
 -        for(MemoryPoolMXBean pool: ManagementFactory.getMemoryPoolMXBeans())
 -            logger.info("{} {}: {}", pool.getName(), pool.getType(), pool.getPeakUsage());
 -        logger.info("Classpath: {}", System.getProperty("java.class.path"));
 -
 -        logger.info("JVM Arguments: {}", ManagementFactory.getRuntimeMXBean().getInputArguments());
 -
 -        // Fail-fast if JNA is not available or failing to initialize properly
 -        // except with -Dcassandra.boot_without_jna=true. See CASSANDRA-6575.
 -        if (!CLibrary.jnaAvailable())
 +        catch (IOException e)
          {
 -            boolean jnaRequired = !Boolean.getBoolean("cassandra.boot_without_jna");
 -
 -            if (jnaRequired)
 -            {
 -                logger.error("JNA failing to initialize properly. Use -Dcassandra.boot_without_jna=true to bootstrap even so.");
 -                System.exit(3);
 -            }
 +            exitOrFail(3, e.getMessage(), e.getCause());
          }
  
 -        CLibrary.tryMlockall();
 -
          maybeInitJmx();
  
          Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4aa859e5/test/unit/org/apache/cassandra/db/DirectoriesTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/DirectoriesTest.java
index f92cecf,4267c1f..008e343
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@@ -36,9 -44,9 +36,10 @@@ import org.apache.cassandra.db.Director
  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.service.DefaultFSErrorHandler;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.io.FSWriteError;
 +import org.apache.cassandra.utils.Pair;
  
  import static org.junit.Assert.assertEquals;
  import static org.junit.Assert.assertFalse;


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

Posted by yu...@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/764f3d3a
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/764f3d3a
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/764f3d3a

Branch: refs/heads/cassandra-3.0
Commit: 764f3d3a1dd4d643cd3178967f14b45d155f4fd6
Parents: a3d0556 4aa859e
Author: Yuki Morishita <yu...@apache.org>
Authored: Thu May 26 09:53:39 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu May 26 09:53:39 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/cassandra/blob/764f3d3a/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 062bb7b,f2276f0..75ce90d
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,44 -1,9 +1,45 @@@
 -2.2.7
 +3.0.7
 + * Prevent OOM failures on SSTable corruption, improve tests for corruption detection (CASSANDRA-9530)
 + * Use CFS.initialDirectories when clearing snapshots (CASSANDRA-11705)
 + * Allow compaction strategies to disable early open (CASSANDRA-11754)
 + * Refactor Materialized View code (CASSANDRA-11475)
 + * Update Java Driver (CASSANDRA-11615)
 +Merged from 2.2:
   * Enable client encryption in sstableloader with cli options (CASSANDRA-11708)
   * Possible memory leak in NIODataInputStream (CASSANDRA-11867)
 - * Fix commit log replay after out-of-order flush completion (CASSANDRA-9669)
   * Add seconds to cqlsh tracing session duration (CASSANDRA-11753)
 - * Prohibit Reverse Counter type as part of the PK (CASSANDRA-9395)
 + * Prohibit Reversed Counter type as part of the PK (CASSANDRA-9395)
 +Merged from 2.1:
++ * Backport CASSANDRA-11578 (CASSANDRA-11750)
 + * Clear out parent repair session if repair coordinator dies (CASSANDRA-11824)
 + * Set default streaming_socket_timeout_in_ms to 24 hours (CASSANDRA-11840)
 + * Do not consider local node a valid source during replace (CASSANDRA-11848)
 + * Add message dropped tasks to nodetool netstats (CASSANDRA-11855)
 + * Avoid holding SSTableReaders for duration of incremental repair (CASSANDRA-11739)
 +
 +
 +3.0.6
 + * Disallow creating view with a static column (CASSANDRA-11602)
 + * Reduce the amount of object allocations caused by the getFunctions methods (CASSANDRA-11593)
 + * Potential error replaying commitlog with smallint/tinyint/date/time types (CASSANDRA-11618)
 + * Fix queries with filtering on counter columns (CASSANDRA-11629)
 + * Improve tombstone printing in sstabledump (CASSANDRA-11655)
 + * Fix paging for range queries where all clustering columns are specified (CASSANDRA-11669)
 + * Don't require HEAP_NEW_SIZE to be set when using G1 (CASSANDRA-11600)
 + * Fix sstabledump not showing cells after tombstone marker (CASSANDRA-11654)
 + * Ignore all LocalStrategy keyspaces for streaming and other related
 +   operations (CASSANDRA-11627)
 + * Ensure columnfilter covers indexed columns for thrift 2i queries (CASSANDRA-11523)
 + * Only open one sstable scanner per sstable (CASSANDRA-11412)
 + * Option to specify ProtocolVersion in cassandra-stress (CASSANDRA-11410)
 + * ArithmeticException in avgFunctionForDecimal (CASSANDRA-11485)
 + * LogAwareFileLister should only use OLD sstable files in current folder to determine disk consistency (CASSANDRA-11470)
 + * Notify indexers of expired rows during compaction (CASSANDRA-11329)
 + * Properly respond with ProtocolError when a v1/v2 native protocol
 +   header is received (CASSANDRA-11464)
 + * Validate that num_tokens and initial_token are consistent with one another (CASSANDRA-10120)
 +Merged from 2.2:
 + * Fix commit log replay after out-of-order flush completion (CASSANDRA-9669)
   * cqlsh: correctly handle non-ascii chars in error messages (CASSANDRA-11626)
   * Exit JVM if JMX server fails to startup (CASSANDRA-11540)
   * Produce a heap dump when exiting on OOM (CASSANDRA-9861)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/764f3d3a/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/FileUtils.java
index 46f2de5,3124fcd..9e81da5
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@@ -25,19 -23,17 +25,17 @@@ import java.nio.charset.StandardCharset
  import java.nio.file.*;
  import java.text.DecimalFormat;
  import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.List;
- 
- import sun.nio.ch.DirectBuffer;
+ import java.util.concurrent.atomic.AtomicReference;
  
 -import org.apache.cassandra.config.Config;
 -import sun.nio.ch.DirectBuffer;
 -
  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;
@@@ -47,10 -42,8 +44,10 @@@ import org.apache.cassandra.utils.JVMSt
  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;
 +
      private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
      private static final double KB = 1024d;
      private static final double MB = 1024*1024d;
@@@ -458,65 -396,30 +456,18 @@@
          dir.deleteOnExit();
      }
  
 -    public static void skipBytesFully(DataInput in, int bytes) throws IOException
 -    {
 -        int n = 0;
 -        while (n < bytes)
 -        {
 -            int skipped = in.skipBytes(bytes - n);
 -            if (skipped == 0)
 -                throw new EOFException("EOF after " + n + " bytes out of " + bytes);
 -            n += skipped;
 -        }
 -    }
 -
      public static void handleCorruptSSTable(CorruptSSTableException e)
      {
-         if (!StorageService.instance.isSetupCompleted())
-             handleStartupFSError(e);
- 
-         JVMStabilityInspector.inspectThrowable(e);
-         switch (DatabaseDescriptor.getDiskFailurePolicy())
-         {
-             case stop_paranoid:
-                 StorageService.instance.stopTransports();
-                 break;
-         }
+         FSErrorHandler handler = fsErrorHandler.get();
+         if (handler != null)
+             handler.handleCorruptSSTable(e);
      }
-     
+ 
      public static void handleFSError(FSError e)
      {
-         if (!StorageService.instance.isSetupCompleted())
-             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;
-         }
+         FSErrorHandler handler = fsErrorHandler.get();
+         if (handler != null)
+             handler.handleFSError(e);
      }
      /**
       * Get the size of a directory in bytes
@@@ -572,54 -474,8 +522,59 @@@
          return false;
      }
  
 +    public static void append(File file, String ... lines)
 +    {
 +        if (file.exists())
 +            write(file, Arrays.asList(lines), StandardOpenOption.APPEND);
 +        else
 +            write(file, Arrays.asList(lines), StandardOpenOption.CREATE);
 +    }
 +
 +    public static void appendAndSync(File file, String ... lines)
 +    {
 +        if (file.exists())
 +            write(file, Arrays.asList(lines), StandardOpenOption.APPEND, StandardOpenOption.SYNC);
 +        else
 +            write(file, Arrays.asList(lines), StandardOpenOption.CREATE, StandardOpenOption.SYNC);
 +    }
 +
 +    public static void replace(File file, String ... lines)
 +    {
 +        write(file, Arrays.asList(lines), StandardOpenOption.TRUNCATE_EXISTING);
 +    }
 +
 +    public static void write(File file, List<String> lines, StandardOpenOption ... options)
 +    {
 +        try
 +        {
 +            Files.write(file.toPath(),
 +                        lines,
 +                        CHARSET,
 +                        options);
 +        }
 +        catch (IOException ex)
 +        {
 +            throw new RuntimeException(ex);
 +        }
 +    }
 +
 +    public static List<String> readLines(File file)
 +    {
 +        try
 +        {
 +            return Files.readAllLines(file.toPath(), CHARSET);
 +        }
 +        catch (IOException ex)
 +        {
 +            if (ex instanceof NoSuchFileException)
 +                return Collections.emptyList();
 +
 +            throw new RuntimeException(ex);
 +        }
 +    }
++
+     public static void setFSErrorHandler(FSErrorHandler handler)
+     {
+         fsErrorHandler.getAndSet(handler);
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/764f3d3a/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/764f3d3a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/DirectoriesTest.java
index f864bbc,008e343..f8d01a8
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@@ -42,8 -36,9 +42,9 @@@ import org.apache.cassandra.io.FSWriteE
  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.io.FSWriteError;
  import org.apache.cassandra.utils.Pair;
  
  import static org.junit.Assert.assertEquals;
@@@ -65,14 -63,10 +66,15 @@@ public class DirectoriesTes
      @BeforeClass
      public static void beforeClass() throws IOException
      {
+         FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
 -        for (String cf : CFS)
 +        for (String table : TABLES)
          {
 -            CFM.add(new CFMetaData(KS, cf, ColumnFamilyType.Standard, null));
 +            UUID tableID = CFMetaData.generateLegacyCfId(KS, table);
 +            CFM.add(CFMetaData.Builder.create(KS, table)
 +                                      .withId(tableID)
 +                                      .addPartitionKey("thekey", UTF8Type.instance)
 +                                      .addClusteringColumn("thecolumn", UTF8Type.instance)
 +                                      .build());
          }
  
          tempDataDir = File.createTempFile("cassandra", "unittest");


[5/6] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by yu...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2


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

Branch: refs/heads/cassandra-2.2
Commit: 4aa859e51a38d0856ba4ee2d0c177d83722271dd
Parents: 63efa07 b851792
Author: Yuki Morishita <yu...@apache.org>
Authored: Thu May 26 09:47:04 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu May 26 09:47:04 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4aa859e5/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 190c2fa,ad9d00c..f2276f0
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,24 -1,5 +1,25 @@@
 -2.1.15
 +2.2.7
 + * Enable client encryption in sstableloader with cli options (CASSANDRA-11708)
 + * Possible memory leak in NIODataInputStream (CASSANDRA-11867)
 + * Fix commit log replay after out-of-order flush completion (CASSANDRA-9669)
 + * Add seconds to cqlsh tracing session duration (CASSANDRA-11753)
 + * Prohibit Reverse Counter type as part of the PK (CASSANDRA-9395)
 + * cqlsh: correctly handle non-ascii chars in error messages (CASSANDRA-11626)
 + * Exit JVM if JMX server fails to startup (CASSANDRA-11540)
 + * Produce a heap dump when exiting on OOM (CASSANDRA-9861)
 + * Avoid read repairing purgeable tombstones on range slices (CASSANDRA-11427)
 + * Restore ability to filter on clustering columns when using a 2i (CASSANDRA-11510)
 + * JSON datetime formatting needs timezone (CASSANDRA-11137)
 + * Fix is_dense recalculation for Thrift-updated tables (CASSANDRA-11502)
 + * Remove unnescessary file existence check during anticompaction (CASSANDRA-11660)
 + * Add missing files to debian packages (CASSANDRA-11642)
 + * Avoid calling Iterables::concat in loops during ModificationStatement::getFunctions (CASSANDRA-11621)
 + * cqlsh: COPY FROM should use regular inserts for single statement batches and
 +   report errors correctly if workers processes crash on initialization (CASSANDRA-11474)
 + * Always close cluster with connection in CqlRecordWriter (CASSANDRA-11553)
 + * Fix slice queries on ordered COMPACT tables (CASSANDRA-10988)
 +Merged from 2.1:
+  * Backport CASSANDRA-11578 (CASSANDRA-11750)
   * Clear out parent repair session if repair coordinator dies (CASSANDRA-11824)
   * Set default streaming_socket_timeout_in_ms to 24 hours (CASSANDRA-11840)
   * Do not consider local node a valid source during replace (CASSANDRA-11848)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4aa859e5/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/util/FileUtils.java
index ce45370,f69ed01..3124fcd
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@@ -17,14 -17,23 +17,15 @@@
   */
  package org.apache.cassandra.io.util;
  
 -import java.io.Closeable;
 -import java.io.DataInput;
 -import java.io.EOFException;
 -import java.io.File;
 -import java.io.FileNotFoundException;
 -import java.io.IOException;
 -import java.io.RandomAccessFile;
 +import java.io.*;
  import java.nio.ByteBuffer;
 -import java.nio.MappedByteBuffer;
 -import java.nio.file.AtomicMoveNotSupportedException;
 -import java.nio.file.Files;
 -import java.nio.file.Path;
 -import java.nio.file.StandardCopyOption;
 +import java.nio.channels.FileChannel;
 +import java.nio.file.*;
  import java.text.DecimalFormat;
  import java.util.Arrays;
+ import java.util.concurrent.atomic.AtomicReference;
  
 +import org.apache.cassandra.config.Config;
  import sun.nio.ch.DirectBuffer;
  
  import org.slf4j.Logger;
@@@ -38,13 -45,9 +37,12 @@@ import org.apache.cassandra.io.FSErrorH
  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
  {
      private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
      private static final double KB = 1024d;
@@@ -487,39 -423,8 +439,43 @@@
          return length;
      }
  
- 
 +    public static void copyTo(DataInput in, OutputStream out, int length) throws IOException
 +    {
 +        byte[] buffer = new byte[64 * 1024];
 +        int copiedBytes = 0;
 +
 +        while (copiedBytes + buffer.length < length)
 +        {
 +            in.readFully(buffer);
 +            out.write(buffer);
 +            copiedBytes += buffer.length;
 +        }
 +
 +        if (copiedBytes < length)
 +        {
 +            int left = length - copiedBytes;
 +            in.readFully(buffer, 0, left);
 +            out.write(buffer, 0, left);
 +        }
 +    }
 +
 +    public static boolean isSubDirectory(File parent, File child) throws IOException
 +    {
 +        parent = parent.getCanonicalFile();
 +        child = child.getCanonicalFile();
 +
 +        File toCheck = child;
 +        while (toCheck != null)
 +        {
 +            if (parent.equals(toCheck))
 +                return true;
 +            toCheck = toCheck.getParentFile();
 +        }
 +        return false;
 +    }
++
+     public static void setFSErrorHandler(FSErrorHandler handler)
+     {
+         fsErrorHandler.getAndSet(handler);
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4aa859e5/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/CassandraDaemon.java
index 7e33e9c,5a1fb14..1e9378b
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@@ -149,32 -146,81 +149,34 @@@ public class CassandraDaemo
       */
      protected void setup()
      {
+         FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
+ 
 -        try
 -        {
 -            logger.info("Hostname: {}", InetAddress.getLocalHost().getHostName());
 -        }
 -        catch (UnknownHostException e1)
 -        {
 -            logger.info("Could not resolve local host");
 -        }
 +        // Delete any failed snapshot deletions on Windows - see CASSANDRA-9658
 +        if (FBUtilities.isWindows())
 +            WindowsFailedSnapshotTracker.deleteOldSnapshots();
  
 -        long now = System.currentTimeMillis();
 -        if (now < EARLIEST_LAUNCH_DATE)
 -        {
 -            logger.error("current machine time is {}, but that is seemingly incorrect. exiting now.", new Date(now));
 -            System.exit(3);
 -        }
 +        logSystemInfo();
 +
 +        CLibrary.tryMlockall();
  
 -        // log warnings for different kinds of sub-optimal JVMs.  tldr use 64-bit Oracle >= 1.6u32
 -        if (!DatabaseDescriptor.hasLargeAddressSpace())
 -            logger.info("32bit JVM detected.  It is recommended to run Cassandra on a 64bit JVM for better performance.");
 -        String javaVersion = System.getProperty("java.version");
 -        String javaVmName = System.getProperty("java.vm.name");
 -        logger.info("JVM vendor/version: {}/{}", javaVmName, javaVersion);
 -        if (javaVmName.contains("OpenJDK"))
 +        try
          {
 -            // There is essentially no QA done on OpenJDK builds, and
 -            // clusters running OpenJDK have seen many heap and load issues.
 -            logger.warn("OpenJDK is not recommended. Please upgrade to the newest Oracle Java release");
 +            startupChecks.verify();
          }
 -        else if (!javaVmName.contains("HotSpot"))
 +        catch (StartupException e)
          {
 -            logger.warn("Non-Oracle JVM detected.  Some features, such as immediate unmap of compacted SSTables, may not work as intended");
 +            exitOrFail(e.returnCode, e.getMessage(), e.getCause());
          }
 -     /*   else
 +
 +        try
          {
 -            String[] java_version = javaVersion.split("_");
 -            String java_major = java_version[0];
 -            int java_minor;
 -            try
 -            {
 -                java_minor = (java_version.length > 1) ? Integer.parseInt(java_version[1]) : 0;
 -            }
 -            catch (NumberFormatException e)
 -            {
 -                // have only seen this with java7 so far but no doubt there are other ways to break this
 -                logger.info("Unable to parse java version {}", Arrays.toString(java_version));
 -                java_minor = 32;
 -            }
 +            SystemKeyspace.snapshotOnVersionChange();
          }
 -     */
 -        logger.info("Heap size: {}/{}", Runtime.getRuntime().totalMemory(), Runtime.getRuntime().maxMemory());
 -        for(MemoryPoolMXBean pool: ManagementFactory.getMemoryPoolMXBeans())
 -            logger.info("{} {}: {}", pool.getName(), pool.getType(), pool.getPeakUsage());
 -        logger.info("Classpath: {}", System.getProperty("java.class.path"));
 -
 -        logger.info("JVM Arguments: {}", ManagementFactory.getRuntimeMXBean().getInputArguments());
 -
 -        // Fail-fast if JNA is not available or failing to initialize properly
 -        // except with -Dcassandra.boot_without_jna=true. See CASSANDRA-6575.
 -        if (!CLibrary.jnaAvailable())
 +        catch (IOException e)
          {
 -            boolean jnaRequired = !Boolean.getBoolean("cassandra.boot_without_jna");
 -
 -            if (jnaRequired)
 -            {
 -                logger.error("JNA failing to initialize properly. Use -Dcassandra.boot_without_jna=true to bootstrap even so.");
 -                System.exit(3);
 -            }
 +            exitOrFail(3, e.getMessage(), e.getCause());
          }
  
 -        CLibrary.tryMlockall();
 -
          maybeInitJmx();
  
          Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4aa859e5/test/unit/org/apache/cassandra/db/DirectoriesTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/DirectoriesTest.java
index f92cecf,4267c1f..008e343
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@@ -36,9 -44,9 +36,10 @@@ import org.apache.cassandra.db.Director
  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.service.DefaultFSErrorHandler;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.io.FSWriteError;
 +import org.apache.cassandra.utils.Pair;
  
  import static org.junit.Assert.assertEquals;
  import static org.junit.Assert.assertFalse;


[2/6] cassandra git commit: Backport CASSANDRA-11578

Posted by yu...@apache.org.
Backport CASSANDRA-11578

patch by yukim; reviewed by Paulo Motta for CASSANDRA-11750


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

Branch: refs/heads/cassandra-2.2
Commit: b851792c4e3ae32b8d863d9079cca6d135f1cf23
Parents: 5dc7414
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed May 18 17:03:39 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu May 26 09:40:40 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, 152 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b851792c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f73db6e..ad9d00c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.15
+ * Backport CASSANDRA-11578 (CASSANDRA-11750)
  * Clear out parent repair session if repair coordinator dies (CASSANDRA-11824)
  * Set default streaming_socket_timeout_in_ms to 24 hours (CASSANDRA-11840)
  * Do not consider local node a valid source during replace (CASSANDRA-11848)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b851792c/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/b851792c/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 3be7c99..f69ed01 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -32,6 +32,7 @@ import java.nio.file.Path;
 import java.nio.file.StandardCopyOption;
 import java.text.DecimalFormat;
 import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicReference;
 
 import sun.nio.ch.DirectBuffer;
 
@@ -39,17 +40,14 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 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;
 
-public class FileUtils
+public final class FileUtils
 {
     private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
     private static final double KB = 1024d;
@@ -59,6 +57,7 @@ public class FileUtils
 
     private static final DecimalFormat df = new DecimalFormat("#.##");
     private static final boolean canCleanDirectBuffers;
+    private static final AtomicReference<FSErrorHandler> fsErrorHandler = new AtomicReference<>();
 
     static
     {
@@ -395,63 +394,16 @@ public class FileUtils
 
     public static void handleCorruptSSTable(CorruptSSTableException e)
     {
-        if (!StorageService.instance.isSetupCompleted())
-            handleStartupFSError(e);
-
-        JVMStabilityInspector.inspectThrowable(e);
-        switch (DatabaseDescriptor.getDiskFailurePolicy())
-        {
-            case stop_paranoid:
-                StorageService.instance.stopTransports();
-                break;
-        }
-    }
-    
-    public static void handleFSError(FSError e)
-    {
-        if (!StorageService.instance.isSetupCompleted())
-            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 handler = fsErrorHandler.get();
+        if (handler != null)
+            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 handler = fsErrorHandler.get();
+        if (handler != null)
+            handler.handleFSError(e);
     }
     /**
      * Get the size of a directory in bytes
@@ -470,4 +422,9 @@ public class FileUtils
         }
         return length;
     }
+
+    public static void setFSErrorHandler(FSErrorHandler handler)
+    {
+        fsErrorHandler.getAndSet(handler);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b851792c/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 a27fa20..5a1fb14 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -148,6 +148,8 @@ public class CassandraDaemon
      */
     protected void setup()
     {
+        FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
+
         try
         {
             logger.info("Hostname: {}", InetAddress.getLocalHost().getHostName());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b851792c/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..88a1fce
--- /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.isSetupCompleted())
+            handleStartupFSError(e);
+
+        JVMStabilityInspector.inspectThrowable(e);
+        switch (DatabaseDescriptor.getDiskFailurePolicy())
+        {
+            case stop_paranoid:
+                StorageService.instance.stopTransports();
+                break;
+        }
+    }
+
+    @Override
+    public void handleFSError(FSError e)
+    {
+        if (!StorageService.instance.isSetupCompleted())
+            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/b851792c/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 b1c51ee..4267c1f 100644
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@ -44,6 +44,7 @@ import org.apache.cassandra.db.Directories.DataDirectory;
 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.service.DefaultFSErrorHandler;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.io.FSWriteError;
 
@@ -65,6 +66,7 @@ public class DirectoriesTest
     @BeforeClass
     public static void beforeClass() throws IOException
     {
+        FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
         for (String cf : CFS)
         {
             CFM.add(new CFMetaData(KS, cf, ColumnFamilyType.Standard, null));


[3/6] cassandra git commit: Backport CASSANDRA-11578

Posted by yu...@apache.org.
Backport CASSANDRA-11578

patch by yukim; reviewed by Paulo Motta for CASSANDRA-11750


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

Branch: refs/heads/cassandra-3.0
Commit: b851792c4e3ae32b8d863d9079cca6d135f1cf23
Parents: 5dc7414
Author: Yuki Morishita <yu...@apache.org>
Authored: Wed May 18 17:03:39 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu May 26 09:40:40 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, 152 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b851792c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f73db6e..ad9d00c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.15
+ * Backport CASSANDRA-11578 (CASSANDRA-11750)
  * Clear out parent repair session if repair coordinator dies (CASSANDRA-11824)
  * Set default streaming_socket_timeout_in_ms to 24 hours (CASSANDRA-11840)
  * Do not consider local node a valid source during replace (CASSANDRA-11848)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b851792c/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/b851792c/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 3be7c99..f69ed01 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -32,6 +32,7 @@ import java.nio.file.Path;
 import java.nio.file.StandardCopyOption;
 import java.text.DecimalFormat;
 import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicReference;
 
 import sun.nio.ch.DirectBuffer;
 
@@ -39,17 +40,14 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 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;
 
-public class FileUtils
+public final class FileUtils
 {
     private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
     private static final double KB = 1024d;
@@ -59,6 +57,7 @@ public class FileUtils
 
     private static final DecimalFormat df = new DecimalFormat("#.##");
     private static final boolean canCleanDirectBuffers;
+    private static final AtomicReference<FSErrorHandler> fsErrorHandler = new AtomicReference<>();
 
     static
     {
@@ -395,63 +394,16 @@ public class FileUtils
 
     public static void handleCorruptSSTable(CorruptSSTableException e)
     {
-        if (!StorageService.instance.isSetupCompleted())
-            handleStartupFSError(e);
-
-        JVMStabilityInspector.inspectThrowable(e);
-        switch (DatabaseDescriptor.getDiskFailurePolicy())
-        {
-            case stop_paranoid:
-                StorageService.instance.stopTransports();
-                break;
-        }
-    }
-    
-    public static void handleFSError(FSError e)
-    {
-        if (!StorageService.instance.isSetupCompleted())
-            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 handler = fsErrorHandler.get();
+        if (handler != null)
+            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 handler = fsErrorHandler.get();
+        if (handler != null)
+            handler.handleFSError(e);
     }
     /**
      * Get the size of a directory in bytes
@@ -470,4 +422,9 @@ public class FileUtils
         }
         return length;
     }
+
+    public static void setFSErrorHandler(FSErrorHandler handler)
+    {
+        fsErrorHandler.getAndSet(handler);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b851792c/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 a27fa20..5a1fb14 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -148,6 +148,8 @@ public class CassandraDaemon
      */
     protected void setup()
     {
+        FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
+
         try
         {
             logger.info("Hostname: {}", InetAddress.getLocalHost().getHostName());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b851792c/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..88a1fce
--- /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.isSetupCompleted())
+            handleStartupFSError(e);
+
+        JVMStabilityInspector.inspectThrowable(e);
+        switch (DatabaseDescriptor.getDiskFailurePolicy())
+        {
+            case stop_paranoid:
+                StorageService.instance.stopTransports();
+                break;
+        }
+    }
+
+    @Override
+    public void handleFSError(FSError e)
+    {
+        if (!StorageService.instance.isSetupCompleted())
+            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/b851792c/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 b1c51ee..4267c1f 100644
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@ -44,6 +44,7 @@ import org.apache.cassandra.db.Directories.DataDirectory;
 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.service.DefaultFSErrorHandler;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.io.FSWriteError;
 
@@ -65,6 +66,7 @@ public class DirectoriesTest
     @BeforeClass
     public static void beforeClass() throws IOException
     {
+        FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
         for (String cf : CFS)
         {
             CFM.add(new CFMetaData(KS, cf, ColumnFamilyType.Standard, null));