Repository: cassandra
Updated Branches:
  refs/heads/trunk d4f2354e4 -> 194bad22f


Add pre-startup checks to detect potential incompatibilities

patch by Sam Tunnicliffe; reviewed by Aleksey Yeschenko for
CASSANDRA-8049


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

Branch: refs/heads/trunk
Commit: 194bad22f71da3007a7f9ab5561d6f211d140c45
Parents: d4f2354
Author: Sam Tunnicliffe <[email protected]>
Authored: Mon Jan 26 14:25:15 2015 +0000
Committer: Aleksey Yeschenko <[email protected]>
Committed: Tue Apr 28 15:18:36 2015 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 build.xml                                       |   4 +
 .../org/apache/cassandra/db/SystemKeyspace.java |  93 +++++-
 .../cassandra/exceptions/StartupException.java  |  39 +++
 .../cassandra/service/CassandraDaemon.java      | 220 ++++----------
 .../apache/cassandra/service/StartupCheck.java  |  47 +++
 .../apache/cassandra/service/StartupChecks.java | 292 +++++++++++++++++++
 .../Standard1/Keyspace1-Standard1-ic-0-Data.db  | Bin 0 -> 530 bytes
 .../Keyspace1-Standard1-ic-0-Digest.sha1        |   1 +
 .../Keyspace1-Standard1-ic-0-Filter.db          | Bin 0 -> 24 bytes
 .../Standard1/Keyspace1-Standard1-ic-0-Index.db | Bin 0 -> 180 bytes
 .../Keyspace1-Standard1-ic-0-Statistics.db      | Bin 0 -> 4361 bytes
 .../Keyspace1-Standard1-ic-0-Summary.db         | Bin 0 -> 92 bytes
 .../Standard1/Keyspace1-Standard1-ic-0-TOC.txt  |   7 +
 .../apache/cassandra/db/SystemKeyspaceTest.java |  72 +++++
 .../cassandra/service/StartupChecksTest.java    | 110 +++++++
 16 files changed, 712 insertions(+), 174 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7502a14..0f0439d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0
+ * Add pre-startup checks to detect potential incompatibilities 
(CASSANDRA-8049)
  * Distinguish between null and unset in protocol v4 (CASSANDRA-7304)
  * Add user/role permissions for user-defined functions (CASSANDRA-7557)
  * Allow cassandra config to be updated to restart daemon without unloading 
classes (CASSANDRA-9046)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 8cc03d5..ba99cd9 100644
--- a/build.xml
+++ b/build.xml
@@ -1203,6 +1203,7 @@
   <target name="testold" depends="build-test" description="Execute unit tests">
     <testmacro suitename="unit" inputdir="${test.unit.src}" 
exclude="**/pig/*.java" timeout="${test.timeout}">
       <jvmarg value="-Dlegacy-sstable-root=${test.data}/legacy-sstables"/>
+      <jvmarg 
value="-Dinvalid-legacy-sstable-root=${test.data}/invalid-legacy-sstables"/>
       <jvmarg value="-Dcorrupt-sstable-root=${test.data}/corrupt-sstables"/>
       <jvmarg 
value="-Dmigration-sstable-root=${test.data}/migration-sstables"/>
       <jvmarg value="-Dcassandra.ring_delay_ms=1000"/>
@@ -1217,6 +1218,7 @@
   <target name="testlist">
     <testmacro suitename="${testlist.name}" inputdir="${test.unit.src}" 
filelist="${test.file.list}" poffset="${testlist.offset}" exclude="**/*.java" 
timeout="${test.timeout}">
       <jvmarg value="-Dlegacy-sstable-root=${test.data}/legacy-sstables"/>
+      <jvmarg 
value="-Dinvalid-legacy-sstable-root=${test.data}/invalid-legacy-sstables"/>
       <jvmarg value="-Dcorrupt-sstable-root=${test.data}/corrupt-sstables"/>
       <jvmarg 
value="-Dmigration-sstable-root=${test.data}/migration-sstables"/>
       <jvmarg value="-Dcassandra.ring_delay_ms=1000"/>
@@ -1244,6 +1246,7 @@
     <testmacro suitename="unit" inputdir="${test.unit.src}" 
exclude="**/pig/*.java" timeout="${test.timeout}">
       <test name="${test.name}" methods="${test.methods}"/>
       <jvmarg value="-Dlegacy-sstable-root=${test.data}/legacy-sstables"/>
+      <jvmarg 
value="-Dinvalid-legacy-sstable-root=${test.data}/invalid-legacy-sstables"/>
       <jvmarg value="-Dcorrupt-sstable-root=${test.data}/corrupt-sstables"/>
       <jvmarg 
value="-Dmigration-sstable-root=${test.data}/migration-sstables"/>
       <jvmarg value="-Dcassandra.ring_delay_ms=1000"/>
@@ -1260,6 +1263,7 @@
     <echo>Compressed config: ${compressed_yaml}</echo>
     <testmacro suitename="unit" inputdir="${test.unit.src}" 
exclude="**/pig/*.java" timeout="${test.timeout}">
       <jvmarg value="-Dlegacy-sstable-root=${test.data}/legacy-sstables"/>
+      <jvmarg 
value="-Dinvalid-legacy-sstable-root=${test.data}/invalid-legacy-sstables"/>
       <jvmarg value="-Dcorrupt-sstable-root=${test.data}/corrupt-sstables"/>
       <jvmarg 
value="-Dmigration-sstable-root=${test.data}/migration-sstables"/>
       <jvmarg value="-Dcassandra.test.compression=true"/>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java 
b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 89ff61b..c9d77f4 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -17,14 +17,13 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.DataInputStream;
-import java.io.IOError;
-import java.io.IOException;
+import java.io.*;
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
-import javax.management.openmbean.*;
+import javax.management.openmbean.OpenDataException;
+import javax.management.openmbean.TabularData;
 
 import com.google.common.base.Function;
 import com.google.common.collect.*;
@@ -68,6 +67,15 @@ public final class SystemKeyspace
 {
     private static final Logger logger = 
LoggerFactory.getLogger(SystemKeyspace.class);
 
+    // Used to indicate that there was a previous version written to the 
legacy (pre 1.2)
+    // system.Versions table, but that we cannot read it. Suffice to say, any 
upgrade should
+    // proceed through 1.2.x before upgrading to the current version.
+    public static final SemanticVersion UNREADABLE_VERSION = new 
SemanticVersion("0.0.0-unknown");
+
+    // Used to indicate that no previous version information was found. When 
encountered, we assume that
+    // Cassandra was not previously installed and we're in the process of 
starting a fresh node.
+    public static final SemanticVersion NULL_VERSION = new 
SemanticVersion("0.0.0-absent");
+
     public static final String NAME = "system";
 
     public static final String HINTS = "hints";
@@ -289,10 +297,21 @@ public final class SystemKeyspace
 
     private static void setupVersion()
     {
-        String req = "INSERT INTO system.%s (key, release_version, 
cql_version, thrift_version, native_protocol_version, data_center, rack, 
partitioner) VALUES (?, ?, ?, ?, ?, ?, ?, ?)";
+        String req = "INSERT INTO system.%s (" +
+                     "  key, " +
+                     "  cluster_name, " +
+                     "  release_version, " +
+                     "  cql_version, " +
+                     "  thrift_version, " +
+                     "  native_protocol_version, " +
+                     "  data_center, " +
+                     "  rack, " +
+                     "  partitioner" +
+                     ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
         IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
         executeOnceInternal(String.format(req, LOCAL),
                             LOCAL,
+                            DatabaseDescriptor.getClusterName(),
                             FBUtilities.getReleaseVersionString(),
                             QueryProcessor.CQL_VERSION.toString(),
                             cassandraConstants.VERSION,
@@ -686,8 +705,6 @@ public final class SystemKeyspace
                 throw new ConfigurationException("Found system keyspace files, 
but they couldn't be loaded!");
 
             // no system files.  this is a new node.
-            req = "INSERT INTO system.%s (key, cluster_name) VALUES ('%s', ?)";
-            executeInternal(String.format(req, LOCAL, LOCAL), 
DatabaseDescriptor.getClusterName());
             return;
         }
 
@@ -1004,6 +1021,68 @@ public final class SystemKeyspace
         availableRanges.truncateBlocking();
     }
 
+    /**
+     * Compare the release version in the system.local table with the one 
included in the distro.
+     * If they don't match, snapshot all tables in the system keyspace. This 
is intended to be
+     * called at startup to create a backup of the system tables during an 
upgrade
+     *
+     * @throws IOException
+     */
+    public static void snapshotOnVersionChange() throws IOException
+    {
+        String previous = getPreviousVersionString();
+        String next = FBUtilities.getReleaseVersionString();
+
+        // if we're restarting after an upgrade, snapshot the system keyspace
+        if (!previous.equals(NULL_VERSION.toString()) && 
!previous.equals(next))
+
+        {
+            logger.info("Detected version upgrade from {} to {}, snapshotting 
system keyspace", previous, next);
+            String snapshotName = 
Keyspace.getTimestampedSnapshotName(String.format("upgrade-%s-%s",
+                                                                               
     previous,
+                                                                               
     next));
+            Keyspace systemKs = Keyspace.open(SystemKeyspace.NAME);
+            systemKs.snapshot(snapshotName, null);
+        }
+    }
+
+    /**
+     * Try to determine what the previous version, if any, was installed on 
this node.
+     * Primary source of truth is the release version in system.local. If the 
previous
+     * version cannot be determined by looking there then either:
+     * * the node never had a C* install before
+     * * the was a very old version (pre 1.2) installed, which did not include 
system.local
+     *
+     * @return either a version read from the system.local table or one of two 
special values
+     * indicating either no previous version (SystemUpgrade.NULL_VERSION) or 
an unreadable,
+     * legacy version (SystemUpgrade.UNREADABLE_VERSION).
+     */
+    private static String getPreviousVersionString()
+    {
+        String req = "SELECT release_version FROM system.%s WHERE key='%s'";
+        UntypedResultSet result = executeInternal(String.format(req, 
SystemKeyspace.LOCAL, SystemKeyspace.LOCAL));
+        if (result.isEmpty() || !result.one().has("release_version"))
+        {
+            // it isn't inconceivable that one might try to upgrade a node 
straight from <= 1.1 to whatever
+            // the current version is. If we couldn't read a previous version 
from system.local we check for
+            // the existence of the legacy system.Versions table. We don't 
actually attempt to read a version
+            // from there, but it informs us that this isn't a completely new 
node.
+            for (File dataDirectory : 
Directories.getKSChildDirectories(SystemKeyspace.NAME))
+            {
+                if (dataDirectory.getName().equals("Versions") && 
dataDirectory.listFiles().length > 0)
+                {
+                    logger.debug("Found unreadable versions info in pre 1.2 
system.Versions table");
+                    return UNREADABLE_VERSION.toString();
+                }
+            }
+
+            // no previous version information found, we can assume that this 
is a new node
+            return NULL_VERSION.toString();
+        }
+        // report back whatever we found in the system table
+        return result.one().getString("release_version");
+    }
+
     private static ByteBuffer rangeToBytes(Range<Token> range)
     {
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/src/java/org/apache/cassandra/exceptions/StartupException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/StartupException.java 
b/src/java/org/apache/cassandra/exceptions/StartupException.java
new file mode 100644
index 0000000..ec4890f
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/StartupException.java
@@ -0,0 +1,39 @@
+/*
+ * 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.exceptions;
+
+/**
+ * An exception thrown during system startup, indicating the environment or 
system
+ * is not in a valid state to startup.
+ */
+public class StartupException extends Exception
+{
+    public final int returnCode;
+
+    public StartupException(int returnCode, String message)
+    {
+        super(message);
+        this.returnCode = returnCode;
+    }
+
+    public StartupException(int returnCode, String message, Throwable cause)
+    {
+        super(message, cause);
+        this.returnCode = returnCode;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/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 d357e92..c19dac8 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -25,7 +25,9 @@ import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.rmi.registry.LocateRegistry;
 import java.rmi.server.RMIServerSocketFactory;
-import java.util.*;
+import java.util.Collections;
+import java.util.Map;
+import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
@@ -34,18 +36,12 @@ import javax.management.remote.JMXConnectorServer;
 import javax.management.remote.JMXServiceURL;
 import javax.management.remote.rmi.RMIConnectorServer;
 
-import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.Uninterruptibles;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.addthis.metrics3.reporter.config.ReporterConfig;
-
-import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
-import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.concurrent.Stage;
-import org.apache.cassandra.concurrent.StageManager;
+import org.apache.cassandra.concurrent.*;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
@@ -53,6 +49,7 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.StartupException;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.util.FileUtils;
@@ -71,74 +68,42 @@ import org.apache.cassandra.utils.*;
 public class CassandraDaemon
 {
     public static final String MBEAN_NAME = 
"org.apache.cassandra.db:type=NativeAccess";
-    public static JMXConnectorServer jmxServer = null;
+    private static JMXConnectorServer jmxServer = null;
 
     private static final Logger logger = 
LoggerFactory.getLogger(CassandraDaemon.class);
 
     private static void maybeInitJmx()
     {
-        String jmxPort = 
System.getProperty("com.sun.management.jmxremote.port");
+        if (System.getProperty("com.sun.management.jmxremote.port") != null)
+            return;
 
+        String jmxPort = System.getProperty("cassandra.jmx.local.port");
         if (jmxPort == null)
-        {
-            logger.warn("JMX is not enabled to receive remote connections. 
Please see cassandra-env.sh for more info.");
-
-            jmxPort = System.getProperty("cassandra.jmx.local.port");
-
-            if (jmxPort == null)
-            {
-                logger.error("cassandra.jmx.local.port missing from 
cassandra-env.sh, unable to start local JMX service." + jmxPort);
-            }
-            else
-            {
-                System.setProperty("java.rmi.server.hostname", 
InetAddress.getLoopbackAddress().getHostAddress());
+            return;
 
-                try
-                {
-                    RMIServerSocketFactory serverFactory = new 
RMIServerSocketFactoryImpl();
-                    LocateRegistry.createRegistry(Integer.valueOf(jmxPort), 
null, serverFactory);
-
-                    StringBuffer url = new StringBuffer();
-                    url.append("service:jmx:");
-                    url.append("rmi://localhost/jndi/");
-                    
url.append("rmi://localhost:").append(jmxPort).append("/jmxrmi");
-                    
-                    Map env = new HashMap();
-                    
env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, serverFactory);
-
-                    jmxServer = new RMIConnectorServer(
-                            new JMXServiceURL(url.toString()),
-                            env,
-                            ManagementFactory.getPlatformMBeanServer()
-                    );
-
-                    jmxServer.start();
-                }
-                catch (IOException e)
-                {
-                    logger.error("Error starting local jmx server: ", e);
-                }
-            }
+        System.setProperty("java.rmi.server.hostname", 
InetAddress.getLoopbackAddress().getHostAddress());
+        RMIServerSocketFactory serverFactory = new 
RMIServerSocketFactoryImpl();
+        Map<String, ?> env = 
Collections.singletonMap(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE,
 serverFactory);
+        try
+        {
+            LocateRegistry.createRegistry(Integer.valueOf(jmxPort), null, 
serverFactory);
+            JMXServiceURL url = new 
JMXServiceURL(String.format("service:jmx:rmi://localhost/jndi/rmi://localhost:%s/jmxrmi",
 jmxPort));
+            jmxServer = new RMIConnectorServer(url, env, 
ManagementFactory.getPlatformMBeanServer());
+            jmxServer.start();
         }
-        else
+        catch (IOException e)
         {
-            logger.info("JMX is enabled to receive remote connections on port: 
" + jmxPort);
+            logger.error("Error starting local jmx server: ", e);
         }
     }
 
     private static final CassandraDaemon instance = new CassandraDaemon();
 
-    /**
-     * The earliest legit timestamp a casandra instance could have ever 
launched.
-     * Date roughly taken from 
http://perspectives.mvdirona.com/2008/07/12/FacebookReleasesCassandraAsOpenSource.aspx
-     * We use this to ensure the system clock is at least somewhat correct at 
startup.
-     */
-    private static final long EARLIEST_LAUNCH_DATE = 1215820800000L;
-
     public Server thriftServer;
     public Server nativeServer;
 
     private final boolean runManaged;
+    protected final StartupChecks startupChecks;
 
     public CassandraDaemon() {
         this(false);
@@ -146,6 +111,7 @@ public class CassandraDaemon
 
     public CassandraDaemon(boolean runManaged) {
         this.runManaged = runManaged;
+        this.startupChecks = new StartupChecks().withDefaultTests();
     }
 
     /**
@@ -155,74 +121,27 @@ public class CassandraDaemon
      */
     protected void setup()
     {
+        logSystemInfo();
+
         try
         {
-            logger.info("Hostname: {}", 
InetAddress.getLocalHost().getHostName());
+            startupChecks.verify();
         }
-        catch (UnknownHostException e1)
+        catch (StartupException e)
         {
-            logger.info("Could not resolve local host");
+            exitOrFail(e.returnCode, e.getMessage(), e.getCause());
         }
 
-        long now = System.currentTimeMillis();
-        if (now < EARLIEST_LAUNCH_DATE)
-        {
-            String msg = String.format("current machine time is %s, but that 
is seemingly incorrect. exiting now.", new Date(now).toString());
-            logger.error(msg);
-            throw new IllegalStateException(msg);
-        }
+        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");
+            SystemKeyspace.snapshotOnVersionChange();
         }
-        else if (!javaVmName.contains("HotSpot"))
+        catch (IOException e)
         {
-            logger.warn("Non-Oracle JVM detected.  Some features, such as 
immediate unmap of compacted SSTables, may not work as intended");
+            exitOrFail(3, e.getMessage(), e.getCause());
         }
-     /*   else
-        {
-            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;
-            }
-        }
-     */
-        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"));
-
-        // 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())
-        {
-            boolean jnaRequired = 
!Boolean.getBoolean("cassandra.boot_without_jna");
-
-            if (jnaRequired)
-            {
-                exitOrFail(3, "JNA failing to initialize properly. Use 
-Dcassandra.boot_without_jna=true to bootstrap even so.");
-            }
-        }
-
-        CLibrary.tryMlockall();
 
         maybeInitJmx();
 
@@ -254,59 +173,6 @@ public class CassandraDaemon
             }
         });
 
-        // check all directories(data, commitlog, saved cache) for existence 
and permission
-        Iterable<String> dirs = 
Iterables.concat(Arrays.asList(DatabaseDescriptor.getAllDataFileLocations()),
-                                                 
Arrays.asList(DatabaseDescriptor.getCommitLogLocation(),
-                                                               
DatabaseDescriptor.getSavedCachesLocation()));
-
-        SigarLibrary sigarLibrary = new SigarLibrary();
-        if (sigarLibrary.initialized())
-            sigarLibrary.warnIfRunningInDegradedMode();
-        else
-            logger.info("Sigar could not be initialized");
-
-        for (String dataDir : dirs)
-        {
-            logger.debug("Checking directory {}", dataDir);
-            File dir = new File(dataDir);
-
-            // check that directories exist.
-            if (!dir.exists())
-            {
-                logger.error("Directory {} doesn't exist", dataDir);
-                // if they don't, failing their creation, stop cassandra.
-                if (!dir.mkdirs())
-                {
-                    exitOrFail(3, "Has no permission to create directory "+ 
dataDir);
-                }
-            }
-            // if directories exist verify their permissions
-            if (!Directories.verifyFullPermissions(dir, dataDir))
-            {
-                // if permissions aren't sufficient, stop cassandra.
-                exitOrFail(3, "Insufficient permissions on directory " + 
dataDir);
-            }
-
-
-        }
-
-        if (CacheService.instance == null) // should never happen
-            throw new RuntimeException("Failed to initialize Cache Service.");
-
-        // check the system keyspace to keep user from shooting self in foot 
by changing partitioner, cluster name, etc.
-        // we do a one-off scrub of the system keyspace first; we can't load 
the list of the rest of the keyspaces,
-        // until system keyspace is opened.
-        for (CFMetaData cfm : 
Schema.instance.getKeyspaceMetaData(SystemKeyspace.NAME).values())
-            ColumnFamilyStore.scrubDataDirectories(cfm);
-        try
-        {
-            SystemKeyspace.checkHealth();
-        }
-        catch (ConfigurationException e)
-        {
-            exitOrFail(100, "Fatal exception during initialization", e);
-        }
-
         // load schema from disk
         Schema.instance.loadFromDisk();
 
@@ -456,6 +322,26 @@ public class CassandraDaemon
         nativeServer = new org.apache.cassandra.transport.Server(nativeAddr, 
nativePort);
     }
 
+    private void logSystemInfo()
+    {
+        try
+        {
+            logger.info("Hostname: {}", 
InetAddress.getLocalHost().getHostName());
+        }
+        catch (UnknownHostException e1)
+        {
+            logger.info("Could not resolve local host");
+        }
+
+        logger.info("JVM vendor/version: {}/{}", 
System.getProperty("java.vm.name"), System.getProperty("java.version"));
+        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"));
+    }
+
     /**
      * Initialize the Cassandra Daemon based on the given <a
      * href="http://commons.apache.org/daemon/jsvc.html";>Commons

http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/src/java/org/apache/cassandra/service/StartupCheck.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StartupCheck.java 
b/src/java/org/apache/cassandra/service/StartupCheck.java
new file mode 100644
index 0000000..4d23098
--- /dev/null
+++ b/src/java/org/apache/cassandra/service/StartupCheck.java
@@ -0,0 +1,47 @@
+/*
+ * 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 org.apache.cassandra.exceptions.StartupException;
+
+/**
+ * A test to determine if the system is in a valid state to start up.
+ * Some implementations may not actually halt startup, but provide
+ * information or advice on tuning and non-fatal environmental issues (e.g. 
like
+ * checking for and warning about suboptimal JVM settings).
+ * Other checks may indicate that they system is not in a correct state to be 
started.
+ * Examples include inability to load JNA when the cassandra.boot_without_jna 
option
+ * is not set, missing or unaccessible data directories, unreadable sstables 
and
+ * misconfiguration of cluster_name in cassandra.yaml.
+ *
+ * The StartupChecks class manages a collection of these tests, which it 
executes
+ * right at the beginning of the server settup process.
+ */
+public interface StartupCheck
+{
+    /**
+     * Run some test to determine whether the system is safe to be started
+     * In the case where a test determines it is not safe to proceed, the
+     * test should log a message regarding the reason for the failure and
+     * ideally the steps required to remedy the problem.
+     *
+     * @throws org.apache.cassandra.exceptions.StartupException if the test 
determines
+     * that the environement or system is not in a safe state to startup
+     */
+    void execute() throws StartupException;
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/src/java/org/apache/cassandra/service/StartupChecks.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StartupChecks.java 
b/src/java/org/apache/cassandra/service/StartupChecks.java
new file mode 100644
index 0000000..f9a1789
--- /dev/null
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@ -0,0 +1,292 @@
+/*
+ * 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 java.io.IOException;
+import java.nio.file.*;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.*;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.StartupException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.utils.*;
+
+/**
+ * Verifies that the system and environment is in a fit state to be started.
+ * Used in CassandraDaemon#setup() to check various settings and invariants.
+ *
+ * Each individual test is modelled as an implementation of StartupCheck, 
these are run
+ * at the start of CassandraDaemon#setup() before any local state is mutated. 
The default
+ * checks are a mix of informational tests (inspectJvmOptions), initialization
+ * (initSigarLibrary, checkCacheServiceInitialization) and invariant checking
+ * (checkValidLaunchDate, checkSystemKeyspaceState, checkSSTablesFormat).
+ *
+ * In addition, if checkSystemKeyspaceState determines that the release 
version has
+ * changed since last startup (i.e. the node has been upgraded) it snapshots 
the system
+ * keyspace to make it easier to back out if necessary.
+ *
+ * If any check reports a failure, then the setup method exits with an error 
(after
+ * logging any output from the tests). If all tests report success, setup can 
continue.
+ * We should be careful in future to ensure anything which mutates local state 
(such as
+ * writing new sstables etc) only happens after we've verified the initial 
setup.
+ */
+public class StartupChecks
+{
+    private static final Logger logger = 
LoggerFactory.getLogger(StartupChecks.class);
+
+    // List of checks to run before starting up. If any test reports failure, 
startup will be halted.
+    private final List<StartupCheck> preFlightChecks = new ArrayList<>();
+
+    // The default set of pre-flight checks to run. Order is somewhat 
significant in that we probably
+    // always want the system keyspace check run last, as this actually loads 
the schema for that
+    // keyspace. All other checks should not require any schema initialization.
+    private final List<StartupCheck> DEFAULT_TESTS = 
ImmutableList.of(checkValidLaunchDate,
+                                                                      
checkJMXPorts,
+                                                                      
inspectJvmOptions,
+                                                                      
checkJnaInitialization,
+                                                                      
initSigarLibrary,
+                                                                      
checkDataDirs,
+                                                                      
checkSSTablesFormat,
+                                                                      
checkSystemKeyspaceState);
+
+    public StartupChecks withDefaultTests()
+    {
+        preFlightChecks.addAll(DEFAULT_TESTS);
+        return this;
+    }
+
+    /**
+     * Add system test to be run before schema is loaded during startup
+     * @param test the system test to include
+     */
+    public StartupChecks withTest(StartupCheck test)
+    {
+        preFlightChecks.add(test);
+        return this;
+    }
+
+    /**
+     * Run the configured tests and return a report detailing the results.
+     * @throws org.apache.cassandra.exceptions.StartupException if any test 
determines that the
+     * system is not in an valid state to startup
+     */
+    public void verify() throws StartupException
+    {
+        for (StartupCheck test : preFlightChecks)
+            test.execute();
+    }
+
+    public static final StartupCheck checkValidLaunchDate = new StartupCheck()
+    {
+        /**
+         * The earliest legit timestamp a casandra instance could have ever 
launched.
+         * Date roughly taken from 
http://perspectives.mvdirona.com/2008/07/12/FacebookReleasesCassandraAsOpenSource.aspx
+         * We use this to ensure the system clock is at least somewhat correct 
at startup.
+         */
+        private static final long EARLIEST_LAUNCH_DATE = 1215820800000L;
+        public void execute() throws StartupException
+        {
+            long now = System.currentTimeMillis();
+            if (now < EARLIEST_LAUNCH_DATE)
+                throw new StartupException(1, String.format("current machine 
time is %s, but that is seemingly incorrect. exiting now.",
+                                                            new 
Date(now).toString()));
+        }
+    };
+
+    public static final StartupCheck checkJMXPorts = new StartupCheck()
+    {
+        public void execute()
+        {
+            String jmxPort = 
System.getProperty("com.sun.management.jmxremote.port");
+            if (jmxPort == null)
+            {
+                logger.warn("JMX is not enabled to receive remote connections. 
Please see cassandra-env.sh for more info.");
+                jmxPort = System.getProperty("cassandra.jmx.local.port");
+                if (jmxPort == null)
+                    logger.error("cassandra.jmx.local.port missing from 
cassandra-env.sh, unable to start local JMX service.");
+            }
+            else
+            {
+                logger.info("JMX is enabled to receive remote connections on 
port: " + jmxPort);
+            }
+        }
+    };
+
+    public static final StartupCheck inspectJvmOptions = new StartupCheck()
+    {
+        public void execute()
+        {
+            // log warnings for different kinds of sub-optimal JVMs.  tldr use 
64-bit Oracle >= 1.6u32
+            if (!DatabaseDescriptor.hasLargeAddressSpace())
+                logger.warn("32bit JVM detected.  It is recommended to run 
Cassandra on a 64bit JVM for better performance.");
+
+            String javaVmName = System.getProperty("java.vm.name");
+            if (javaVmName.contains("OpenJDK"))
+            {
+                // 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");
+            }
+            else if (!javaVmName.contains("HotSpot"))
+            {
+                logger.warn("Non-Oracle JVM detected.  Some features, such as 
immediate unmap of compacted SSTables, may not work as intended");
+            }
+        }
+    };
+
+    public static final StartupCheck checkJnaInitialization = new 
StartupCheck()
+    {
+        public void execute() throws StartupException
+        {
+            // 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())
+            {
+                boolean jnaRequired = 
!Boolean.getBoolean("cassandra.boot_without_jna");
+
+                if (jnaRequired)
+                    throw new StartupException(3, "JNA failing to initialize 
properly. " +
+                                                  "Use 
-Dcassandra.boot_without_jna=true to bootstrap even so.");
+            }
+        }
+    };
+
+    public static final StartupCheck initSigarLibrary = new StartupCheck()
+    {
+        public void execute()
+        {
+            new SigarLibrary().warnIfRunningInDegradedMode();
+        }
+    };
+
+    public static final StartupCheck checkDataDirs = new StartupCheck()
+    {
+        public void execute() throws StartupException
+        {
+            // check all directories(data, commitlog, saved cache) for 
existence and permission
+            Iterable<String> dirs = 
Iterables.concat(Arrays.asList(DatabaseDescriptor.getAllDataFileLocations()),
+                                                     
Arrays.asList(DatabaseDescriptor.getCommitLogLocation(),
+                                                                   
DatabaseDescriptor.getSavedCachesLocation()));
+            for (String dataDir : dirs)
+            {
+                logger.debug("Checking directory {}", dataDir);
+                File dir = new File(dataDir);
+
+                // check that directories exist.
+                if (!dir.exists())
+                {
+                    logger.error("Directory {} doesn't exist", dataDir);
+                    // if they don't, failing their creation, stop cassandra.
+                    if (!dir.mkdirs())
+                        throw new StartupException(3, "Has no permission to 
create directory "+ dataDir);
+                }
+
+                // if directories exist verify their permissions
+                if (!Directories.verifyFullPermissions(dir, dataDir))
+                    throw new StartupException(3, "Insufficient permissions on 
directory " + dataDir);
+
+            }
+        }
+    };
+
+    public static final StartupCheck checkSSTablesFormat = new StartupCheck()
+    {
+        public void execute() throws StartupException
+        {
+            final Set<String> invalid = new HashSet<>();
+            FileVisitor<Path> sstableVisitor = new SimpleFileVisitor<Path>()
+            {
+                public FileVisitResult visitFile(Path file, 
BasicFileAttributes attrs) throws IOException
+                {
+                    try
+                    {
+                        if 
(!Descriptor.fromFilename(file.toString()).isCompatible())
+                            invalid.add(file.toString());
+                    }
+                    catch (Exception e)
+                    {
+                        invalid.add(file.toString());
+                    }
+                    return FileVisitResult.CONTINUE;
+                }
+
+                public FileVisitResult preVisitDirectory(Path dir, 
BasicFileAttributes attrs) throws IOException
+                {
+                    String name = dir.getFileName().toString();
+                    return (name.equals("snapshots") || name.equals("backups"))
+                           ? FileVisitResult.SKIP_SUBTREE
+                           : FileVisitResult.CONTINUE;
+                }
+            };
+
+            for (String dataDir : DatabaseDescriptor.getAllDataFileLocations())
+            {
+                try
+                {
+                    Files.walkFileTree(Paths.get(dataDir), sstableVisitor);
+                }
+                catch (IOException e)
+                {
+                    throw new StartupException(3, "Unable to verify sstable 
files on disk", e);
+                }
+            }
+
+            if (!invalid.isEmpty())
+                throw new StartupException(3, String.format("Detected 
unreadable sstables %s, please check " +
+                                                            "NEWS.txt and 
ensure that you have upgraded through " +
+                                                            "all required 
intermediate versions, running " +
+                                                            "upgradesstables",
+                                                            
Joiner.on(",").join(invalid)));
+
+        }
+    };
+
+    public static final StartupCheck checkSystemKeyspaceState = new 
StartupCheck()
+    {
+        public void execute() throws StartupException
+        {
+            // check the system keyspace to keep user from shooting self in 
foot by changing partitioner, cluster name, etc.
+            // we do a one-off scrub of the system keyspace first; we can't 
load the list of the rest of the keyspaces,
+            // until system keyspace is opened.
+            Keyspace systemKs = Keyspace.open(SystemKeyspace.NAME);
+            for (CFMetaData cfm : 
Schema.instance.getKeyspaceMetaData(SystemKeyspace.NAME).values())
+                ColumnFamilyStore.scrubDataDirectories(cfm);
+
+            try
+            {
+                SystemKeyspace.checkHealth();
+            }
+            catch (ConfigurationException e)
+            {
+                throw new StartupException(100, "Fatal exception during 
initialization", e);
+            }
+        }
+    };
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Data.db
----------------------------------------------------------------------
diff --git 
a/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Data.db
 
b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Data.db
new file mode 100644
index 0000000..98d3f41
Binary files /dev/null and 
b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Data.db
 differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Digest.sha1
----------------------------------------------------------------------
diff --git 
a/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Digest.sha1
 
b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Digest.sha1
new file mode 100644
index 0000000..470b056
--- /dev/null
+++ 
b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Digest.sha1
@@ -0,0 +1 @@
+9ee805b905aa147afe14d4f37f5ed3be3af53c72  Keyspace1-legacyleveled-ic-0-Data.db
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Filter.db
----------------------------------------------------------------------
diff --git 
a/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Filter.db
 
b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Filter.db
new file mode 100644
index 0000000..c63729b
Binary files /dev/null and 
b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Filter.db
 differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Index.db
----------------------------------------------------------------------
diff --git 
a/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Index.db
 
b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Index.db
new file mode 100644
index 0000000..6603018
Binary files /dev/null and 
b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Index.db
 differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Statistics.db
----------------------------------------------------------------------
diff --git 
a/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Statistics.db
 
b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Statistics.db
new file mode 100644
index 0000000..5ed9ce0
Binary files /dev/null and 
b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Statistics.db
 differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Summary.db
----------------------------------------------------------------------
diff --git 
a/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Summary.db
 
b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Summary.db
new file mode 100644
index 0000000..c1c8fd8
Binary files /dev/null and 
b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Summary.db
 differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-TOC.txt
----------------------------------------------------------------------
diff --git 
a/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-TOC.txt
 
b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-TOC.txt
new file mode 100644
index 0000000..6baaf14
--- /dev/null
+++ 
b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-TOC.txt
@@ -0,0 +1,7 @@
+Filter.db
+Summary.db
+Data.db
+Digest.sha1
+Index.db
+TOC.txt
+Statistics.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java 
b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
index 25bb584..b0101bc 100644
--- a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
@@ -17,16 +17,24 @@
  */
 package org.apache.cassandra.db;
 
+import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.*;
 
 import org.junit.Test;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.dht.ByteOrderedPartitioner.BytesToken;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.SemanticVersion;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 public class SystemKeyspaceTest
 {
@@ -69,4 +77,68 @@ public class SystemKeyspaceTest
         UUID secondId = SystemKeyspace.getLocalHostId();
         assert firstId.equals(secondId) : String.format("%s != %s%n", 
firstId.toString(), secondId.toString());
     }
+
+    @Test
+    public void snapshotSystemKeyspaceIfUpgrading() throws IOException
+    {
+        // First, check that in the absence of any previous installed version, 
we don't create snapshots
+        for (ColumnFamilyStore cfs : 
Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStores())
+            cfs.clearUnsafe();
+        Keyspace.clearSnapshot(null, SystemKeyspace.NAME);
+
+        SystemKeyspace.snapshotOnVersionChange();
+        assertTrue(getSystemSnapshotFiles().isEmpty());
+
+        // now setup system.local as if we're upgrading from a previous version
+        SemanticVersion next = getCurrentReleaseVersion();
+        setupReleaseVersion(new SemanticVersion(String.format("%s.%s.%s", 
next.major - 1, next.minor, next.patch)));
+        Keyspace.clearSnapshot(null, SystemKeyspace.NAME);
+        assertTrue(getSystemSnapshotFiles().isEmpty());
+
+        // Compare versions again & verify that snapshots were created for all 
tables in the system ks
+        SystemKeyspace.snapshotOnVersionChange();
+        assertEquals(SystemKeyspace.definition().cfMetaData().size(), 
getSystemSnapshotFiles().size());
+
+        // clear out the snapshots & set the previous recorded version equal 
to the latest, we shouldn't
+        // see any new snapshots created this time.
+        Keyspace.clearSnapshot(null, SystemKeyspace.NAME);
+        setupReleaseVersion(getCurrentReleaseVersion());
+
+        SystemKeyspace.snapshotOnVersionChange();
+        assertTrue(getSystemSnapshotFiles().isEmpty());
+    }
+
+    private SemanticVersion getCurrentReleaseVersion()
+    {
+        return new SemanticVersion(FBUtilities.getReleaseVersionString());
+    }
+
+    private Set<String> getSystemSnapshotFiles()
+    {
+        Set<String> snapshottedTableNames = new HashSet<>();
+        for (ColumnFamilyStore cfs : 
Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStores())
+        {
+            if (!cfs.getSnapshotDetails().isEmpty())
+                snapshottedTableNames.add(cfs.getColumnFamilyName());
+        }
+        return snapshottedTableNames;
+    }
+
+    private void setupReleaseVersion(SemanticVersion version)
+    {
+        // besides the release_version, we also need to insert the 
cluster_name or the check
+        // in SystemKeyspace.checkHealth were we verify it matches 
DatabaseDescriptor will fail
+        QueryProcessor.executeInternal(String.format("INSERT INTO 
system.local(key, release_version, cluster_name) " +
+                                                     "VALUES ('local', '%s', 
'%s')",
+                                                     version,
+                                                     
DatabaseDescriptor.getClusterName()));
+        String r = readLocalVersion();
+        assertEquals(String.format("Expected %s, got %s", version, r), 
version.toString(), r);
+    }
+
+    private String readLocalVersion()
+    {
+        UntypedResultSet rs = QueryProcessor.executeInternal("SELECT 
release_version FROM system.local WHERE key='local'");
+        return rs.isEmpty() || !rs.one().has("release_version") ? null : 
rs.one().getString("release_version");
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/194bad22/test/unit/org/apache/cassandra/service/StartupChecksTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/StartupChecksTest.java 
b/test/unit/org/apache/cassandra/service/StartupChecksTest.java
new file mode 100644
index 0000000..834191a
--- /dev/null
+++ b/test/unit/org/apache/cassandra/service/StartupChecksTest.java
@@ -0,0 +1,110 @@
+/*
+ * 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 java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.exceptions.StartupException;
+import org.apache.cassandra.io.util.FileUtils;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class StartupChecksTest
+{
+    public static final String INVALID_LEGACY_SSTABLE_ROOT_PROP = 
"invalid-legacy-sstable-root";
+    StartupChecks startupChecks;
+
+    @BeforeClass
+    public static void setupServer()
+    {
+        SchemaLoader.prepareServer();
+    }
+
+    @Before
+    public void setup()
+    {
+        for (ColumnFamilyStore cfs : 
Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStores())
+            cfs.clearUnsafe();
+        for (File dataDir : 
Directories.getKSChildDirectories(SystemKeyspace.NAME))
+            FileUtils.deleteRecursive(dataDir);
+
+        startupChecks = new StartupChecks();
+    }
+
+    @Test
+    public void failStartupIfInvalidSSTablesFound() throws Exception
+    {
+        startupChecks = 
startupChecks.withTest(StartupChecks.checkSSTablesFormat);
+
+        File dataDir = new 
File(DatabaseDescriptor.getAllDataFileLocations()[0]);
+        Path sstableDir = Paths.get(dataDir.getAbsolutePath(), "Keyspace1", 
"Standard1");
+        Files.createDirectories(sstableDir);
+        copyInvalidLegacySSTables(sstableDir);
+
+        verifyFailure(startupChecks, "Detected unreadable sstables");
+
+        // we should ignore invalid sstables in a snapshots directory
+        FileUtils.deleteRecursive(sstableDir.toFile());
+        Path snapshotDir = sstableDir.resolve("snapshots");
+        Files.createDirectories(snapshotDir);
+        copyInvalidLegacySSTables(snapshotDir); startupChecks.verify();
+
+        // and in a backups directory
+        FileUtils.deleteRecursive(sstableDir.toFile());
+        Path backupDir = sstableDir.resolve("backups");
+        Files.createDirectories(backupDir);
+        copyInvalidLegacySSTables(backupDir);
+        startupChecks.verify();
+    }
+
+    private void copyInvalidLegacySSTables(Path targetDir) throws IOException
+    {
+        File legacySSTableRoot = 
Paths.get(System.getProperty(INVALID_LEGACY_SSTABLE_ROOT_PROP),
+                                           "Keyspace1",
+                                           "Standard1").toFile();
+        for (File f : legacySSTableRoot.listFiles())
+            Files.copy(f.toPath(), targetDir.resolve(f.getName()));
+
+    }
+
+
+    private void verifyFailure(StartupChecks tests, String message)
+    {
+        try
+        {
+            tests.verify();
+            fail("Expected a startup exception but none was thrown");
+        }
+        catch (StartupException e)
+        {
+            assertTrue(e.getMessage().contains(message));
+        }
+    }
+}

Reply via email to