You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2013/08/08 01:05:31 UTC

[04/12] git commit: add --migrate optionto sstableupgrade and sstablescrub patch by Tyler Hobbs; reviewed by Jeremiah Jordan for CASSANDRA-5831

add --migrate optionto sstableupgrade and sstablescrub
patch by Tyler Hobbs; reviewed by Jeremiah Jordan for CASSANDRA-5831


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

Branch: refs/heads/trunk
Commit: 7d45d2bde3724a15a9e0f68bb047a41853ccd340
Parents: 27943b1
Author: Jonathan Ellis <jb...@apache.org>
Authored: Wed Aug 7 18:03:29 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Wed Aug 7 18:03:29 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/db/SystemTable.java    | 52 +++++++++++++++-----
 .../cassandra/tools/StandaloneScrubber.java     | 19 ++++++-
 .../cassandra/tools/StandaloneUpgrader.java     | 22 ++++++++-
 4 files changed, 80 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7d45d2bd/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 25c0217..859e7f9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 1.2.9
+ * add --migrate option to sstableupgrade and sstablescrub (CASSANDRA-5831)
  * fix bulk-loading compressed sstables (CASSANDRA-5820)
  * (Hadoop) fix quoting in CqlPagingRecordReader and CqlRecordWriter 
    (CASSANDRA-5824)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7d45d2bd/src/java/org/apache/cassandra/db/SystemTable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemTable.java b/src/java/org/apache/cassandra/db/SystemTable.java
index 4c74df5..318b34d 100644
--- a/src/java/org/apache/cassandra/db/SystemTable.java
+++ b/src/java/org/apache/cassandra/db/SystemTable.java
@@ -147,6 +147,7 @@ public class SystemTable
         ColumnFamilyStore oldStatusCfs = table.getColumnFamilyStore(OLD_STATUS_CF);
         if (oldStatusCfs.getSSTables().size() > 0)
         {
+            logger.info("Old system data found in {}.{}; migrating to new format in {}.{}", Table.SYSTEM_KS, OLD_STATUS_CF, Table.SYSTEM_KS, LOCAL_CF);
             SortedSet<ByteBuffer> cols = new TreeSet<ByteBuffer>(BytesType.instance);
             cols.add(ByteBufferUtil.bytes("ClusterName"));
             cols.add(ByteBufferUtil.bytes("Token"));
@@ -428,24 +429,51 @@ public class SystemTable
             ex.initCause(err);
             throw ex;
         }
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(LOCAL_CF);
 
-        String req = "SELECT cluster_name FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = processInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
+        String savedClusterName;
 
-        if (result.isEmpty() || !result.one().has("cluster_name"))
+        // See if there is still data in System.LocationInfo, indicating that the system data has not yet been
+        // upgraded by SystemTable.upgradeSystemData()
+        ColumnFamilyStore oldStatusCfs = table.getColumnFamilyStore(OLD_STATUS_CF);
+        if (oldStatusCfs.getSSTables().size() > 0)
+        {
+            logger.debug("Detected system data in {}.{}, checking saved cluster name", Table.SYSTEM_KS, OLD_STATUS_CF);
+            SortedSet<ByteBuffer> cols = new TreeSet<ByteBuffer>(BytesType.instance);
+            cols.add(ByteBufferUtil.bytes("ClusterName"));
+            QueryFilter filter = QueryFilter.getNamesFilter(decorate(ByteBufferUtil.bytes("L")), new QueryPath(OLD_STATUS_CF), cols);
+            ColumnFamily oldCf = oldStatusCfs.getColumnFamily(filter);
+            try
+            {
+                savedClusterName = ByteBufferUtil.string(oldCf.getColumn(ByteBufferUtil.bytes("ClusterName")).value());
+            }
+            catch (CharacterCodingException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+        else
         {
-            // this is a brand new node
-            if (!cfs.getSSTables().isEmpty())
-                throw new ConfigurationException("Found system table files, but they couldn't be loaded!");
+            ColumnFamilyStore cfs = table.getColumnFamilyStore(LOCAL_CF);
 
-            // no system files.  this is a new node.
-            req = "INSERT INTO system.%s (key, cluster_name) VALUES ('%s', '%s')";
-            processInternal(String.format(req, LOCAL_CF, LOCAL_KEY, DatabaseDescriptor.getClusterName()));
-            return;
+            String req = "SELECT cluster_name FROM system.%s WHERE key='%s'";
+            UntypedResultSet result = processInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
+
+            if (result.isEmpty() || !result.one().has("cluster_name"))
+            {
+
+                // this is a brand new node
+                if (!cfs.getSSTables().isEmpty())
+                    throw new ConfigurationException("Found system table 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', '%s')";
+                processInternal(String.format(req, LOCAL_CF, LOCAL_KEY, DatabaseDescriptor.getClusterName()));
+                return;
+            }
+
+            savedClusterName = result.one().getString("cluster_name");
         }
 
-        String savedClusterName = result.one().getString("cluster_name");
         if (!DatabaseDescriptor.getClusterName().equals(savedClusterName))
             throw new ConfigurationException("Saved cluster name " + savedClusterName + " != configured name " + DatabaseDescriptor.getClusterName());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7d45d2bd/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index 0542756..d59a7d1 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -49,15 +49,30 @@ public class StandaloneScrubber
     private static final String DEBUG_OPTION  = "debug";
     private static final String HELP_OPTION  = "help";
     private static final String MANIFEST_CHECK_OPTION  = "manifest-check";
+    private static final String MIGRATE_OPTION  = "migrate";
 
     public static void main(String args[]) throws IOException
     {
         Options options = Options.parseArgs(args);
         try
         {
+            OutputHandler handler = new OutputHandler.SystemOutput(options.verbose, options.debug);
+
             // Migrate sstables from pre-#2749 to the correct location
             if (Directories.sstablesNeedsMigration())
+            {
+                if (!options.migrate)
+                {
+                    System.err.println("Detected a pre-1.1 data directory layout.  For this tool to work, a migration " +
+                                       "must be performed to the 1.1+ format for directories and filenames.  Re-run " +
+                                       TOOL_NAME + " with the --" + MIGRATE_OPTION + " option to automatically " +
+                                       "migrate *all* keyspaces and column families to the new layout.");
+                    System.exit(1);
+                }
+                handler.output("Detected a pre-1.1 data directory layout. All keyspace and column family directories " +
+                               "will be migrated to the 1.1+ format.");
                 Directories.migrateSSTables();
+            }
 
             // load keyspace descriptions.
             DatabaseDescriptor.loadSchemas();
@@ -72,7 +87,6 @@ public class StandaloneScrubber
             ColumnFamilyStore cfs = table.getColumnFamilyStore(options.cfName);
             String snapshotName = "pre-scrub-" + System.currentTimeMillis();
 
-            OutputHandler handler = new OutputHandler.SystemOutput(options.verbose, options.debug);
             Directories.SSTableLister lister = cfs.directories.sstableLister().skipTemporary(true);
 
             List<SSTableReader> sstables = new ArrayList<SSTableReader>();
@@ -184,6 +198,7 @@ public class StandaloneScrubber
         public boolean debug;
         public boolean verbose;
         public boolean manifestCheckOnly;
+        public boolean migrate;
 
         private Options(String tableName, String cfName)
         {
@@ -222,6 +237,7 @@ public class StandaloneScrubber
                 opts.debug = cmd.hasOption(DEBUG_OPTION);
                 opts.verbose = cmd.hasOption(VERBOSE_OPTION);
                 opts.manifestCheckOnly = cmd.hasOption(MANIFEST_CHECK_OPTION);
+                opts.migrate = cmd.hasOption(MIGRATE_OPTION);
 
                 return opts;
             }
@@ -246,6 +262,7 @@ public class StandaloneScrubber
             options.addOption("v",  VERBOSE_OPTION,        "verbose output");
             options.addOption("h",  HELP_OPTION,           "display this help message");
             options.addOption("m",  MANIFEST_CHECK_OPTION, "only check and repair the leveled manifest, without actually scrubbing the sstables");
+            options.addOption(null, MIGRATE_OPTION,        "convert directory layout and filenames to 1.1+ structure");
             return options;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7d45d2bd/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
index 357e99c..16bf588 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
@@ -47,12 +47,30 @@ public class StandaloneUpgrader
     private static final String TOOL_NAME = "sstableupgrade";
     private static final String DEBUG_OPTION  = "debug";
     private static final String HELP_OPTION  = "help";
+    private static final String MIGRATE_OPTION  = "migrate";
 
     public static void main(String args[]) throws IOException
     {
         Options options = Options.parseArgs(args);
         try
         {
+            OutputHandler handler = new OutputHandler.SystemOutput(false, options.debug);
+            // Migrate sstables from pre-#2749 to the correct location
+            if (Directories.sstablesNeedsMigration())
+            {
+                if (!options.migrate)
+                {
+                    System.err.println("Detected a pre-1.1 data directory layout.  For this tool to work, a migration " +
+                                       "must be performed to the 1.1+ format for directories and filenames.  Re-run " +
+                                       TOOL_NAME + " with the --" + MIGRATE_OPTION + " option to automatically " +
+                                       "migrate *all* keyspaces and column families to the new layout.");
+                    System.exit(1);
+                }
+                handler.output("Detected a pre-1.1 data directory layout. All keyspace and column family directories " +
+                               "will be migrated to the 1.1+ format.");
+                Directories.migrateSSTables();
+            }
+
             // load keyspace descriptions.
             DatabaseDescriptor.loadSchemas();
 
@@ -64,7 +82,6 @@ public class StandaloneUpgrader
             Table table = Table.openWithoutSSTables(options.keyspace);
             ColumnFamilyStore cfs = table.getColumnFamilyStore(options.cf);
 
-            OutputHandler handler = new OutputHandler.SystemOutput(false, options.debug);
             Directories.SSTableLister lister = cfs.directories.sstableLister();
             if (options.snapshot != null)
                 lister.onlyBackups(true).snapshots(options.snapshot);
@@ -137,6 +154,7 @@ public class StandaloneUpgrader
         public final String snapshot;
 
         public boolean debug;
+        public boolean migrate;
 
         private Options(String keyspace, String cf, String snapshot)
         {
@@ -176,6 +194,7 @@ public class StandaloneUpgrader
                 Options opts = new Options(keyspace, cf, snapshot);
 
                 opts.debug = cmd.hasOption(DEBUG_OPTION);
+                opts.migrate = cmd.hasOption(MIGRATE_OPTION);
 
                 return opts;
             }
@@ -197,6 +216,7 @@ public class StandaloneUpgrader
         {
             CmdLineOptions options = new CmdLineOptions();
             options.addOption(null, DEBUG_OPTION,          "display stack traces");
+            options.addOption(null, MIGRATE_OPTION,        "convert directory layout and filenames to 1.1+ structure");
             options.addOption("h",  HELP_OPTION,           "display this help message");
             return options;
         }