You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/04/09 08:02:10 UTC

svn commit: r1465894 - in /hbase/trunk/hbase-server/src: main/java/org/apache/hadoop/hbase/catalog/ main/java/org/apache/hadoop/hbase/master/ test/java/org/apache/hadoop/hbase/catalog/

Author: stack
Date: Tue Apr  9 06:02:10 2013
New Revision: 1465894

URL: http://svn.apache.org/r1465894
Log:
HBASE-8045 Fix .META. migration after HBASE-3171

Modified:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/MetaMigrationConvertingToPB.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/MetaMigrationConvertingToPB.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/MetaMigrationConvertingToPB.java?rev=1465894&r1=1465893&r2=1465894&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/MetaMigrationConvertingToPB.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/catalog/MetaMigrationConvertingToPB.java Tue Apr  9 06:02:10 2013
@@ -34,8 +34,8 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * A tool to migrate the data stored in ROOT and META tables to pbuf serialization.
- * Supports migrating from 0.92.x and 0.94.x to 0.96.x for the catalog tables.
+ * A tool to migrate the data stored in META table to pbuf serialization.
+ * Supports migrating from 0.92.x and 0.94.x to 0.96.x for the catalog table.
  * @deprecated will be removed for the major release after 0.96.
  */
 @Deprecated
@@ -121,55 +121,29 @@ public class MetaMigrationConvertingToPB
   }
 
   /**
-   * Update ROOT and META to newer version, converting writable serialization to PB, if
-   * it is needed.
+   * Converting writable serialization to PB, if it is needed.
    * @param services MasterServices to get a handle on master
    * @return num migrated rows
    * @throws IOException or RuntimeException if something goes wrong
    */
-  public static long updateRootAndMetaIfNecessary(final MasterServices services)
+  public static long updateMetaIfNecessary(final MasterServices services)
   throws IOException {
-    if (isMetaHRIUpdated(services.getCatalogTracker())) {
-      LOG.info("ROOT/META already up-to date with PB serialization");
+    if (isMetaTableUpdated(services.getCatalogTracker())) {
+      LOG.info("META already up-to date with PB serialization");
       return 0;
     }
-    LOG.info("ROOT/META has Writable serializations, migrating ROOT and META to PB serialization");
+    LOG.info("META has Writable serializations, migrating META to PB serialization");
     try {
-      long rows = updateRootAndMeta(services);
-      LOG.info("ROOT and META updated with PB serialization. Total rows updated: " + rows);
+      long rows = updateMeta(services);
+      LOG.info("META updated with PB serialization. Total rows updated: " + rows);
       return rows;
     } catch (IOException e) {
-      LOG.warn("Update ROOT/META with PB serialization failed." +
-        "Master startup aborted.");
+      LOG.warn("Update META with PB serialization failed." + "Master startup aborted.");
       throw e;
     }
   }
 
   /**
-   * Update ROOT and META to newer version, converting writable serialization to PB
-   * @return  num migrated rows
-   */
-  static long updateRootAndMeta(final MasterServices masterServices)
-      throws IOException {
-    long rows = updateRoot(masterServices);
-    rows += updateMeta(masterServices);
-    return rows;
-  }
-
-  /**
-   * Update ROOT rows, converting writable serialization to PB
-   * @return num migrated rows
-   */
-  static long updateRoot(final MasterServices masterServices)
-  throws IOException {
-    LOG.info("Starting update of ROOT");
-    ConvertToPBMetaVisitor v = new ConvertToPBMetaVisitor(masterServices);
-    MetaReader.fullScan(masterServices.getCatalogTracker(), v, null);
-    LOG.info("Finished update of ROOT. Total rows updated:" + v.numMigratedRows);
-    return v.numMigratedRows;
-  }
-
-  /**
    * Update META rows, converting writable serialization to PB
    * @return num migrated rows
    */
@@ -177,45 +151,27 @@ public class MetaMigrationConvertingToPB
     LOG.info("Starting update of META");
     ConvertToPBMetaVisitor v = new ConvertToPBMetaVisitor(masterServices);
     MetaReader.fullScan(masterServices.getCatalogTracker(), v);
-    //updateRootWithMetaMigrationStatus(masterServices.getCatalogTracker());
     LOG.info("Finished update of META. Total rows updated:" + v.numMigratedRows);
     return v.numMigratedRows;
   }
 
   /**
-   * Update the version flag in -ROOT-.
-   * @param catalogTracker the catalog tracker
-   * @throws IOException
-   */
-  static void updateRootWithMetaMigrationStatus(final CatalogTracker catalogTracker)
-      throws IOException {
-    Put p = new Put(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
-    p.add(HConstants.CATALOG_FAMILY, HConstants.META_VERSION_QUALIFIER,
-        Bytes.toBytes(HConstants.META_VERSION));
-    // TODO so wrong
-    //MetaEditor.putToRootTable(catalogTracker, p);
-    LOG.info("Updated -ROOT- meta version=" + HConstants.META_VERSION);
-  }
-
-  /**
    * @param catalogTracker the catalog tracker
    * @return True if the meta table has been migrated.
    * @throws IOException
    */
-  static boolean isMetaHRIUpdated(final CatalogTracker catalogTracker) throws IOException {
+  static boolean isMetaTableUpdated(final CatalogTracker catalogTracker) throws IOException {
     List<Result> results = MetaReader.fullScanOfMeta(catalogTracker);
     if (results == null || results.isEmpty()) {
-      LOG.info(".META. is not migrated");
-      return false;
+      LOG.info(".META. doesn't have any entries to update.");
+      return true;
+    }
+    for (Result r : results) {
+      byte[] value = r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
+      if (!isMigrated(value)) {
+        return false;
+      }
     }
-    // Presume only the one result because we only support one meta region.
-    Result r = results.get(0);
-    byte [] value = r.getValue(HConstants.CATALOG_FAMILY,
-        HConstants.META_VERSION_QUALIFIER);
-    short version = value == null || value.length <= 0? -1: Bytes.toShort(value);
-
-    boolean migrated = version >= HConstants.META_VERSION;
-    LOG.info("Meta version=" + version + "; migrated=" + migrated);
-    return migrated;
+    return true;
   }
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1465894&r1=1465893&r2=1465894&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Tue Apr  9 06:02:10 2013
@@ -764,12 +764,12 @@ Server {
     if (!assignMeta(status)) return;
     enableServerShutdownHandler();
 
-    // Update meta with new PB serialization if required. i.e migrate all HRI
-    // to PB serialization in meta and update the status in ROOT. This must happen
-    // before we assign all user regions or else the assignment will fail.
+    // Update meta with new PB serialization if required. i.e migrate all HRI to PB serialization
+    // in meta. This must happen before we assign all user regions or else the assignment will 
+    // fail.
     // TODO: Remove this after 0.96, when we do 0.98.
     org.apache.hadoop.hbase.catalog.MetaMigrationConvertingToPB
-      .updateRootAndMetaIfNecessary(this);
+      .updateMetaIfNecessary(this);
 
     this.balancer.setMasterServices(this);
     // Fix up assignment manager status

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java?rev=1465894&r1=1465893&r2=1465894&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaMigrationConvertingToPB.java Tue Apr  9 06:02:10 2013
@@ -56,7 +56,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 /**
- * TODO reenable the tests once a migration path is figured without ROOT
  * Test migration that changes HRI serialization into PB. Tests by bringing up a cluster from actual
  * data from a 0.92 cluster, as well as manually downgrading and then upgrading the META info.
  * @deprecated Remove after 0.96
@@ -169,16 +168,16 @@ public class TestMetaMigrationConverting
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  //@Test
+  @Test
   public void testMetaUpdatedFlagInROOT() throws Exception {
     HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
     boolean metaUpdated = MetaMigrationConvertingToPB.
-      isMetaHRIUpdated(master.getCatalogTracker());
+      isMetaTableUpdated(master.getCatalogTracker());
     assertEquals(true, metaUpdated);
     verifyMetaRowsAreUpdated(master.getCatalogTracker());
   }
 
-  //@Test
+  @Test
   public void testMetaMigration() throws Exception {
     LOG.info("Starting testMetaMigration");
     final byte [] FAMILY = Bytes.toBytes("family");
@@ -207,7 +206,7 @@ public class TestMetaMigrationConverting
 
     // Assert that the flag in ROOT is updated to reflect the correct status
     boolean metaUpdated =
-        MetaMigrationConvertingToPB.isMetaHRIUpdated(
+        MetaMigrationConvertingToPB.isMetaTableUpdated(
         TEST_UTIL.getMiniHBaseCluster().getMaster().getCatalogTracker());
     assertEquals(true, metaUpdated);
     verifyMetaRowsAreUpdated(ct);
@@ -225,7 +224,7 @@ public class TestMetaMigrationConverting
    * rows and migrate any pending rows at startup.
    * @throws Exception
    */
-  //@Test
+  @Test
   public void testMasterCrashDuringMetaMigration() throws Exception {
     final byte[] FAMILY = Bytes.toBytes("family");
     HTableDescriptor htd = new HTableDescriptor("testMasterCrashDuringMetaMigration");
@@ -245,13 +244,13 @@ public class TestMetaMigrationConverting
     LOG.info("Meta Print completed.testUpdatesOnMetaWithLegacyHRI");
 
     long numMigratedRows =
-        MetaMigrationConvertingToPB.updateRootAndMetaIfNecessary(
+        MetaMigrationConvertingToPB.updateMetaIfNecessary(
             TEST_UTIL.getHBaseCluster().getMaster());
     assertEquals(numMigratedRows, 10);
 
     // Assert that the flag in ROOT is updated to reflect the correct status
     boolean metaUpdated = MetaMigrationConvertingToPB.
-      isMetaHRIUpdated(TEST_UTIL.getMiniHBaseCluster().getMaster().getCatalogTracker());
+      isMetaTableUpdated(TEST_UTIL.getMiniHBaseCluster().getMaster().getCatalogTracker());
     assertEquals(true, metaUpdated);
 
     verifyMetaRowsAreUpdated(ct);