You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by mb...@apache.org on 2013/03/26 23:19:12 UTC

svn commit: r1461344 - in /hbase/branches/0.95/hbase-server/src: main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java

Author: mbertozzi
Date: Tue Mar 26 22:19:12 2013
New Revision: 1461344

URL: http://svn.apache.org/r1461344
Log:
HBASE-8199 Eliminate exception for ExportSnapshot against the null table snapshot (Julian Zhou)

Modified:
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java?rev=1461344&r1=1461343&r2=1461344&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java Tue Mar 26 22:19:12 2013
@@ -628,9 +628,13 @@ public final class ExportSnapshot extend
     // The snapshot references must be copied before the files otherwise the files gets removed
     // by the HFileArchiver, since they have no references.
     try {
-      if (!runCopyJob(inputRoot, outputRoot, files, verifyChecksum,
-          filesUser, filesGroup, filesMode, mappers)) {
-        throw new ExportSnapshotException("Snapshot export failed!");
+      if (files.size() == 0) {
+        LOG.warn("There are 0 store file to be copied. There may be no data in the table.");
+      } else {
+        if (!runCopyJob(inputRoot, outputRoot, files, verifyChecksum,
+            filesUser, filesGroup, filesMode, mappers)) {
+          throw new ExportSnapshotException("Snapshot export failed!");
+        }
       }
 
       // Step 3 - Rename fs2:/.snapshot/.tmp/<snapshot> fs2:/.snapshot/<snapshot>

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java?rev=1461344&r1=1461343&r2=1461344&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java Tue Mar 26 22:19:12 2013
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MediumTests;
@@ -70,6 +71,7 @@ public class TestExportSnapshot {
 
   private final static byte[] FAMILY = Bytes.toBytes("cf");
 
+  private byte[] emptySnapshotName;
   private byte[] snapshotName;
   private byte[] tableName;
   private HBaseAdmin admin;
@@ -99,11 +101,19 @@ public class TestExportSnapshot {
     long tid = System.currentTimeMillis();
     tableName = Bytes.toBytes("testtb-" + tid);
     snapshotName = Bytes.toBytes("snaptb0-" + tid);
+    emptySnapshotName = Bytes.toBytes("emptySnaptb0-" + tid);
 
     // create Table
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(new HColumnDescriptor(FAMILY));
     admin.createTable(htd, null);
+
+    // Take an empty snapshot
+    admin.disableTable(tableName);
+    admin.snapshot(emptySnapshotName, tableName);
+    admin.enableTable(tableName);
+
+    // Add some rows
     HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
     TEST_UTIL.loadTable(table, FAMILY);
 
@@ -115,7 +125,11 @@ public class TestExportSnapshot {
 
   @After
   public void tearDown() throws Exception {
-    this.admin.close();
+    admin.disableTable(tableName);
+    admin.deleteSnapshot(snapshotName);
+    admin.deleteSnapshot(emptySnapshotName);
+    admin.deleteTable(tableName);
+    admin.close();
   }
 
   /**
@@ -160,6 +174,19 @@ public class TestExportSnapshot {
    */
   @Test
   public void testExportFileSystemState() throws Exception {
+    testExportFileSystemState(tableName, snapshotName, 2);
+  }
+
+  @Test
+  public void testEmptyExportFileSystemState() throws Exception {
+    testExportFileSystemState(tableName, emptySnapshotName, 1);
+  }
+
+  /**
+   * Test ExportSnapshot
+   */
+  private void testExportFileSystemState(final byte[] tableName, final byte[] snapshotName,
+      int filesExpected) throws Exception {
     Path copyDir = TEST_UTIL.getDataTestDir("export-" + System.currentTimeMillis());
     URI hdfsUri = FileSystem.get(TEST_UTIL.getConfiguration()).getUri();
     FileSystem fs = FileSystem.get(copyDir.toUri(), new Configuration());
@@ -174,7 +201,7 @@ public class TestExportSnapshot {
 
     // Verify File-System state
     FileStatus[] rootFiles = fs.listStatus(copyDir);
-    assertEquals(2, rootFiles.length);
+    assertEquals(filesExpected, rootFiles.length);
     for (FileStatus fileStatus: rootFiles) {
       String name = fileStatus.getPath().getName();
       assertTrue(fileStatus.isDir());
@@ -254,4 +281,3 @@ public class TestExportSnapshot {
     return files;
   }
 }
-