You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2016/10/24 22:57:26 UTC

[1/3] hbase git commit: HBASE-16827 Backup.proto refactoring (Vladimir Rodionov)

Repository: hbase
Updated Branches:
  refs/heads/HBASE-7912 33d76f13f -> 42e56586d


http://git-wip-us.apache.org/repos/asf/hbase/blob/42e56586/hbase-protocol/src/main/protobuf/Backup.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Backup.proto b/hbase-protocol/src/main/protobuf/Backup.proto
index 2b3feeb..876626d 100644
--- a/hbase-protocol/src/main/protobuf/Backup.proto
+++ b/hbase-protocol/src/main/protobuf/Backup.proto
@@ -9,7 +9,7 @@
  *
  *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * Unless required by applicable law or agreed to in writing, software
+ * Unless optional 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
@@ -27,70 +27,42 @@ option optimize_for = SPEED;
 
 import "HBase.proto";
 
-/*enum FullTableBackupState {
-  PRE_SNAPSHOT_TABLE = 1;
-  SNAPSHOT_TABLES = 2;
-  SNAPSHOT_COPY = 3;
-  BACKUP_COMPLETE = 4;
-}
-
-enum IncrementalTableBackupState {
-  PREPARE_INCREMENTAL = 1;
-  INCREMENTAL_COPY = 2;
-  INCR_BACKUP_COMPLETE = 3;
-}
-
-message SnapshotTableStateData {
-  required TableName table = 1;
-  required string snapshotName = 2;
-}
-*/
 enum BackupType {
   FULL = 0;
   INCREMENTAL = 1;
 }
 
-message BackupImage {
-  required string backup_id = 1;
-  required BackupType backup_type = 2;
-  required string root_dir = 3;
-  repeated TableName table_list = 4;
-  required uint64 start_ts = 5;
-  required uint64 complete_ts = 6;
-  repeated BackupImage ancestors = 7; 
-}
-
 message ServerTimestamp {
-  required string server = 1;
-  required uint64 timestamp = 2;
+  optional ServerName server = 1;
+  optional uint64 timestamp = 2;
 }
 
 message TableServerTimestamp {
-  required TableName table = 1;
+  optional TableName table = 1;
   repeated ServerTimestamp server_timestamp = 2;
 }
 
-message BackupManifest {
-  required string version = 1;
-  required string backup_id = 2;
-  required BackupType type = 3;
+message BackupImage {
+  optional string backup_id = 1;
+  optional BackupType backup_type = 2;
+  optional string root_dir = 3;
   repeated TableName table_list = 4;
-  required uint64 start_ts = 5;
-  required uint64 complete_ts = 6;
-  repeated TableServerTimestamp tst_map = 7;
-  repeated BackupImage dependent_backup_image = 8;
+  optional uint64 start_ts = 5;
+  optional uint64 complete_ts = 6;
+  repeated BackupImage ancestors = 7; 
+  repeated TableServerTimestamp tst_map = 8;  
 }
 
 message TableBackupStatus {
-  required TableName table = 1;
-  required string target_dir = 2;
+  optional TableName table = 1;
+  optional string target_dir = 2;
   optional string snapshot = 3; 	
 }
 
 message BackupInfo {
-  required string backup_id = 1;
-  required BackupType type = 2;
-  required string target_root_dir = 3;
+  optional string backup_id = 1;
+  optional BackupType type = 2;
+  optional string target_root_dir = 3;
   optional BackupState state = 4;
   optional BackupPhase phase = 5;
   optional string failed_message = 6;
@@ -99,8 +71,8 @@ message BackupInfo {
   optional uint64  end_ts = 9;
   optional uint32 progress = 10; 
   optional string job_id = 11;
-  required uint32 workers_number = 12;
-  required uint64 bandwidth = 13;
+  optional uint32 workers_number = 12;
+  optional uint64 bandwidth = 13;
   
   enum BackupState {
     WAITING = 0;
@@ -119,9 +91,3 @@ message BackupInfo {
     STORE_MANIFEST = 5;
   } 
 }
-/*
-message BackupProcContext {
-  required BackupInfo ctx = 1;
-  repeated ServerTimestamp server_timestamp = 2;
-}
-*/

http://git-wip-us.apache.org/repos/asf/hbase/blob/42e56586/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
index 878d23b..9ea9f9d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
@@ -36,19 +36,17 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupInfo;
 import org.apache.hadoop.hbase.backup.BackupType;
 import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 
-import com.google.protobuf.InvalidProtocolBufferException;
-
 
 /**
  * Backup manifest Contains all the meta data of a backup image. The manifest info will be bundled
@@ -64,9 +62,6 @@ public class BackupManifest {
   // manifest file name
   public static final String MANIFEST_FILE_NAME = ".backup.manifest";
 
-  // manifest file version, current is 1.0
-  public static final String MANIFEST_VERSION = "1.0";
-
   // backup image, the dependency graph is made up by series of backup images
 
   public static class BackupImage implements Comparable<BackupImage> {
@@ -78,7 +73,8 @@ public class BackupManifest {
     private long startTs;
     private long completeTs;
     private ArrayList<BackupImage> ancestors;
-
+    private HashMap<TableName, HashMap<String, Long>> incrTimeRanges;
+    
     public BackupImage() {
       super();
     }
@@ -114,6 +110,7 @@ public class BackupManifest {
       for(BackupProtos.BackupImage img: ancestorList) {
         image.addAncestor(fromProto(img));
       }
+      image.setIncrTimeRanges(loadIncrementalTimestampMap(im));
       return image;
     }
 
@@ -138,10 +135,62 @@ public class BackupManifest {
           builder.addAncestors(im.toProto());
         }
       }
-
+      
+      setIncrementalTimestampMap(builder);      
       return builder.build();
     }
 
+    
+    private static HashMap<TableName, HashMap<String, Long>> 
+        loadIncrementalTimestampMap(BackupProtos.BackupImage proto) {
+      List<BackupProtos.TableServerTimestamp> list = proto.getTstMapList();
+      
+      HashMap<TableName, HashMap<String, Long>> incrTimeRanges = 
+          new HashMap<TableName, HashMap<String, Long>>();
+      if(list == null || list.size() == 0) return incrTimeRanges;
+      for(BackupProtos.TableServerTimestamp tst: list){
+        TableName tn = ProtobufUtil.toTableName(tst.getTable());
+        HashMap<String, Long> map = incrTimeRanges.get(tn);
+        if(map == null){
+          map = new HashMap<String, Long>();
+          incrTimeRanges.put(tn, map);
+        }
+        List<BackupProtos.ServerTimestamp> listSt = tst.getServerTimestampList();
+        for(BackupProtos.ServerTimestamp stm: listSt) {
+          ServerName sn = ProtobufUtil.toServerName(stm.getServer());
+          map.put(sn.getHostname() +":" + sn.getPort(), stm.getTimestamp());
+        }
+      }
+      return incrTimeRanges;
+    }
+
+    
+    private void setIncrementalTimestampMap(BackupProtos.BackupImage.Builder builder) {
+      if (this.incrTimeRanges == null) {
+        return;
+      }
+      for (Entry<TableName, HashMap<String,Long>> entry: this.incrTimeRanges.entrySet()) {
+        TableName key = entry.getKey();
+        HashMap<String, Long> value = entry.getValue();
+        BackupProtos.TableServerTimestamp.Builder tstBuilder =
+            BackupProtos.TableServerTimestamp.newBuilder();
+        tstBuilder.setTable(ProtobufUtil.toProtoTableName(key));
+
+        for (Map.Entry<String, Long> entry2 : value.entrySet()) {
+          String s = entry2.getKey();
+          BackupProtos.ServerTimestamp.Builder stBuilder = BackupProtos.ServerTimestamp.newBuilder();
+          HBaseProtos.ServerName.Builder snBuilder = HBaseProtos.ServerName.newBuilder();
+          ServerName sn = ServerName.parseServerName(s);
+          snBuilder.setHostName(sn.getHostname());
+          snBuilder.setPort(sn.getPort());
+          stBuilder.setServer(snBuilder.build());        
+          stBuilder.setTimestamp(entry2.getValue());
+          tstBuilder.addServerTimestamp(stBuilder.build());
+        }
+        builder.addTstMap(tstBuilder.build());
+      }
+    } 
+    
     public String getBackupId() {
       return backupId;
     }
@@ -253,10 +302,15 @@ public class BackupManifest {
       }
       return hash;
     }
-  }
 
-  // manifest version
-  private String version = MANIFEST_VERSION;
+    public HashMap<TableName, HashMap<String, Long>> getIncrTimeRanges() {
+      return incrTimeRanges;
+    }
+
+    public void setIncrTimeRanges(HashMap<TableName, HashMap<String, Long>> incrTimeRanges) {
+      this.incrTimeRanges = incrTimeRanges;
+    }
+  }
 
   // hadoop hbase configuration
   protected Configuration config = null;
@@ -290,7 +344,8 @@ public class BackupManifest {
   private Map<TableName, HashMap<String, Long>> incrTimeRanges;
 
   // dependency of this backup, including all the dependent images to do PIT recovery
-  private Map<String, BackupImage> dependency;
+  //private Map<String, BackupImage> dependency;
+  private BackupImage backupImage;
   
   /**
    * Construct manifest for a ongoing backup.
@@ -306,6 +361,8 @@ public class BackupManifest {
     this.startTs = backupCtx.getStartTs();
     this.completeTs = backupCtx.getEndTs();
     this.loadTableList(backupCtx.getTableNames());
+    this.backupImage = new BackupImage(this.backupId, this.type, this.rootDir, tableList, this.startTs,
+     this.completeTs);
   }
   
   
@@ -326,6 +383,8 @@ public class BackupManifest {
     List<TableName> tables = new ArrayList<TableName>();
     tables.add(table);
     this.loadTableList(tables);
+    this.backupImage = new BackupImage(this.backupId, this.type, this.rootDir, tableList, this.startTs,
+      this.completeTs);
   }
 
   /**
@@ -372,15 +431,13 @@ public class BackupManifest {
           long len = subFile.getLen();
           byte[] pbBytes = new byte[(int) len];
           in.readFully(pbBytes);
-          BackupProtos.BackupManifest proto = null;
+          BackupProtos.BackupImage proto = null;
           try{
-            proto = parseFrom(pbBytes);
+            proto = BackupProtos.BackupImage.parseFrom(pbBytes);
           } catch(Exception e){
             throw new BackupException(e);
           }
-          this.version = proto.getVersion();
-          this.backupId = proto.getBackupId();
-          this.type = BackupType.valueOf(proto.getType().name());
+          this.backupImage = BackupImage.fromProto(proto);
           // Here the parameter backupDir is where the manifest file is.
           // There should always be a manifest file under:
           // backupRootDir/namespace/table/backupId/.backup.manifest
@@ -392,13 +449,12 @@ public class BackupManifest {
           } else {
             this.rootDir = p.getParent().getParent().toString();
           }
-
-          loadTableList(proto);
-          this.startTs = proto.getStartTs();
-          this.completeTs = proto.getCompleteTs();
-          loadIncrementalTimestampMap(proto);
-          loadDependency(proto);
-          //TODO: merge will be implemented by future jira
+          this.backupId = this.backupImage.getBackupId();
+          this.startTs = this.backupImage.getStartTs();
+          this.completeTs = this.backupImage.getCompleteTs();
+          this.type = this.backupImage.getType();
+          this.tableList = (ArrayList<TableName>)this.backupImage.getTableNames();
+          this.incrTimeRanges = this.backupImage.getIncrTimeRanges();
           LOG.debug("Loaded manifest instance from manifest file: "
               + BackupClientUtil.getPath(subFile.getPath()));
           return;
@@ -411,50 +467,6 @@ public class BackupManifest {
       throw new BackupException(e.getMessage());
     }
   }
-  
-  private void loadIncrementalTimestampMap(BackupProtos.BackupManifest proto) {
-    List<BackupProtos.TableServerTimestamp> list = proto.getTstMapList();
-    if(list == null || list.size() == 0) return;
-    this.incrTimeRanges = new HashMap<TableName, HashMap<String, Long>>();
-    for(BackupProtos.TableServerTimestamp tst: list){
-      TableName tn = ProtobufUtil.toTableName(tst.getTable());
-      HashMap<String, Long> map = this.incrTimeRanges.get(tn);
-      if(map == null){
-        map = new HashMap<String, Long>();
-        this.incrTimeRanges.put(tn, map);
-      }
-      List<BackupProtos.ServerTimestamp> listSt = tst.getServerTimestampList();
-      for(BackupProtos.ServerTimestamp stm: listSt) {
-        map.put(stm.getServer(), stm.getTimestamp());
-      }
-    }
-  }
-
-  private void loadDependency(BackupProtos.BackupManifest proto) {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("load dependency for: "+proto.getBackupId());
-    }
-
-    dependency = new HashMap<String, BackupImage>();
-    List<BackupProtos.BackupImage> list = proto.getDependentBackupImageList();
-    for (BackupProtos.BackupImage im : list) {
-      BackupImage bim = BackupImage.fromProto(im);
-      if(im.getBackupId() != null){
-        dependency.put(im.getBackupId(), bim);
-      } else{
-        LOG.warn("Load dependency for backup manifest: "+ backupId+ 
-          ". Null backup id in dependent image");
-      }
-    }
-  }
-
-  private void loadTableList(BackupProtos.BackupManifest proto) {
-    this.tableList = new ArrayList<TableName>();
-    List<HBaseProtos.TableName> list = proto.getTableListList();
-    for (HBaseProtos.TableName name: list) {
-      this.tableList.add(ProtobufUtil.toTableName(name));
-    }
-  }
 
   public BackupType getType() {
     return type;
@@ -498,8 +510,7 @@ public class BackupManifest {
    */
 
   public void store(Configuration conf) throws BackupException {
-    byte[] data = toByteArray();
-
+    byte[] data = backupImage.toProto().toByteArray();
     // write the file, overwrite if already exist
     Path manifestFilePath =
         new Path(new Path((this.tableBackupDir != null ? this.tableBackupDir : this.logBackupDir))
@@ -517,87 +528,11 @@ public class BackupManifest {
   }
 
   /**
-   * Protobuf serialization
-   * @return The filter serialized using pb
-   */
-  public byte[] toByteArray() {
-    BackupProtos.BackupManifest.Builder builder = BackupProtos.BackupManifest.newBuilder();
-    builder.setVersion(this.version);
-    builder.setBackupId(this.backupId);
-    builder.setType(BackupProtos.BackupType.valueOf(this.type.name()));
-    setTableList(builder);
-    builder.setStartTs(this.startTs);
-    builder.setCompleteTs(this.completeTs);
-    setIncrementalTimestampMap(builder);
-    setDependencyMap(builder);
-    return builder.build().toByteArray();
-  }
-
-  private void setIncrementalTimestampMap(BackupProtos.BackupManifest.Builder builder) {
-    if (this.incrTimeRanges == null) {
-      return;
-    }
-    for (Entry<TableName, HashMap<String,Long>> entry: this.incrTimeRanges.entrySet()) {
-      TableName key = entry.getKey();
-      HashMap<String, Long> value = entry.getValue();
-      BackupProtos.TableServerTimestamp.Builder tstBuilder =
-          BackupProtos.TableServerTimestamp.newBuilder();
-      tstBuilder.setTable(ProtobufUtil.toProtoTableName(key));
-
-      for (Map.Entry<String, Long> entry2 : value.entrySet()) {
-        String s = entry2.getKey();
-        BackupProtos.ServerTimestamp.Builder stBuilder = BackupProtos.ServerTimestamp.newBuilder();
-        stBuilder.setServer(s);
-        stBuilder.setTimestamp(entry2.getValue());
-        tstBuilder.addServerTimestamp(stBuilder.build());
-      }
-      builder.addTstMap(tstBuilder.build());
-    }
-  }
-
-  private void setDependencyMap(BackupProtos.BackupManifest.Builder builder) {
-    for (BackupImage image: getDependency().values()) {
-      builder.addDependentBackupImage(image.toProto());
-    }
-  }
-
-  private void setTableList(BackupProtos.BackupManifest.Builder builder) {
-    for(TableName name: tableList){
-      builder.addTableList(ProtobufUtil.toProtoTableName(name));
-    }
-  }
-
-  /**
-   * Parse protobuf from byte array
-   * @param pbBytes A pb serialized BackupManifest instance
-   * @return An instance of  made from <code>bytes</code>
-   * @throws DeserializationException
-   */
-  private static BackupProtos.BackupManifest parseFrom(final byte[] pbBytes)
-      throws DeserializationException {
-    BackupProtos.BackupManifest proto;
-    try {
-      proto = BackupProtos.BackupManifest.parseFrom(pbBytes);
-    } catch (InvalidProtocolBufferException e) {
-      throw new DeserializationException(e);
-    }
-    return proto;
-  }
-
-  /**
-   * Get manifest file version
-   * @return version
-   */
-  public String getVersion() {
-    return version;
-  }
-
-  /**
    * Get this backup image.
    * @return the backup image.
    */
   public BackupImage getBackupImage() {
-    return this.getDependency().get(this.backupId);
+    return backupImage;
   }
 
   /**
@@ -605,25 +540,7 @@ public class BackupManifest {
    * @param image The direct dependent backup image
    */
   public void addDependentImage(BackupImage image) {
-    this.getDependency().get(this.backupId).addAncestor(image);
-    this.setDependencyMap(this.getDependency(), image);
-  }
-
-
-
-  /**
-   * Get all dependent backup images. The image of this backup is also contained.
-   * @return The dependent backup images map
-   */
-  public Map<String, BackupImage> getDependency() {
-    if (this.dependency == null) {
-      this.dependency = new HashMap<String, BackupImage>();
-      LOG.debug(this.rootDir + " " + this.backupId + " " + this.type);
-      this.dependency.put(this.backupId,
-        new BackupImage(this.backupId, this.type, this.rootDir, tableList, this.startTs,
-            this.completeTs));
-    }
-    return this.dependency;
+    this.backupImage.addAncestor(image);
   }
 
   /**
@@ -632,9 +549,9 @@ public class BackupManifest {
    */
   public void setIncrTimestampMap(HashMap<TableName, HashMap<String, Long>> incrTimestampMap) {
     this.incrTimeRanges = incrTimestampMap;
+    this.backupImage.setIncrTimeRanges(incrTimestampMap);
   }
 
-
   public Map<TableName, HashMap<String, Long>> getIncrTimestampMap() {
     if (this.incrTimeRanges == null) {
       this.incrTimeRanges = new HashMap<TableName, HashMap<String, Long>>();
@@ -642,7 +559,6 @@ public class BackupManifest {
     return this.incrTimeRanges;
   }
 
-
   /**
    * Get the image list of this backup for restore in time order.
    * @param reverse If true, then output in reverse order, otherwise in time order from old to new
@@ -650,7 +566,8 @@ public class BackupManifest {
    */
   public ArrayList<BackupImage> getRestoreDependentList(boolean reverse) {
     TreeMap<Long, BackupImage> restoreImages = new TreeMap<Long, BackupImage>();
-    for (BackupImage image : this.getDependency().values()) {
+    restoreImages.put(backupImage.startTs, backupImage);
+    for (BackupImage image : backupImage.getAncestors()) {
       restoreImages.put(Long.valueOf(image.startTs), image);
     }
     return new ArrayList<BackupImage>(reverse ? (restoreImages.descendingMap().values())
@@ -696,23 +613,6 @@ public class BackupManifest {
     return tableImageList;
   }
 
-
-  /**
-   * Recursively set the dependency map of the backup images.
-   * @param map The dependency map
-   * @param image The backup image
-   */
-  private void setDependencyMap(Map<String, BackupImage> map, BackupImage image) {
-    if (image == null) {
-      return;
-    } else {
-      map.put(image.getBackupId(), image);
-      for (BackupImage img : image.getAncestors()) {
-        setDependencyMap(map, img);
-      }
-    }
-  }
-
   /**
    * Check whether backup image1 could cover backup image2 or not.
    * @param image1 backup image 1

http://git-wip-us.apache.org/repos/asf/hbase/blob/42e56586/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
index 72c28d5..c19bc97 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupInfo;
 import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
@@ -55,6 +56,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 
 /**
  * This class provides 'hbase:backup' table API
@@ -498,7 +500,11 @@ public final class BackupSystemTable implements Closeable {
 
     for (Entry<String, Long> entry : map.entrySet()) {
       BackupProtos.ServerTimestamp.Builder builder = BackupProtos.ServerTimestamp.newBuilder();
-      builder.setServer(entry.getKey());
+      HBaseProtos.ServerName.Builder snBuilder = HBaseProtos.ServerName.newBuilder();
+      ServerName sn = ServerName.parseServerName(entry.getKey());
+      snBuilder.setHostName(sn.getHostname());
+      snBuilder.setPort(sn.getPort());
+      builder.setServer(snBuilder.build());
       builder.setTimestamp(entry.getValue());
       tstBuilder.addServerTimestamp(builder.build());
     }
@@ -511,7 +517,8 @@ public final class BackupSystemTable implements Closeable {
     HashMap<String, Long> map = new HashMap<String, Long>();
     List<BackupProtos.ServerTimestamp> list = proto.getServerTimestampList();
     for (BackupProtos.ServerTimestamp st : list) {
-      map.put(st.getServer(), st.getTimestamp());
+      ServerName sn = ProtobufUtil.toServerName(st.getServer());
+      map.put(sn.getHostname()+":"+sn.getPort(), st.getTimestamp());
     }
     return map;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/42e56586/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
index 75832d3..3488786 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
@@ -269,9 +269,9 @@ public class TestBackupSystemTable {
 
     HashMap<String, Long> rsTimestampMap = new HashMap<String, Long>();
 
-    rsTimestampMap.put("rs1", 100L);
-    rsTimestampMap.put("rs2", 101L);
-    rsTimestampMap.put("rs3", 103L);
+    rsTimestampMap.put("rs1:100", 100L);
+    rsTimestampMap.put("rs2:100", 101L);
+    rsTimestampMap.put("rs3:100", 103L);
 
     table.writeRegionServerLogTimestamp(tables, rsTimestampMap, "root");
 
@@ -282,9 +282,9 @@ public class TestBackupSystemTable {
     for (TableName t : tables) {
       HashMap<String, Long> rstm = result.get(t);
       assertNotNull(rstm);
-      assertEquals(rstm.get("rs1"), new Long(100L));
-      assertEquals(rstm.get("rs2"), new Long(101L));
-      assertEquals(rstm.get("rs3"), new Long(103L));
+      assertEquals(rstm.get("rs1:100"), new Long(100L));
+      assertEquals(rstm.get("rs2:100"), new Long(101L));
+      assertEquals(rstm.get("rs3:100"), new Long(103L));
     }
 
     Set<TableName> tables1 = new TreeSet<>();
@@ -295,9 +295,9 @@ public class TestBackupSystemTable {
 
     HashMap<String, Long> rsTimestampMap1 = new HashMap<String, Long>();
 
-    rsTimestampMap1.put("rs1", 200L);
-    rsTimestampMap1.put("rs2", 201L);
-    rsTimestampMap1.put("rs3", 203L);
+    rsTimestampMap1.put("rs1:100", 200L);
+    rsTimestampMap1.put("rs2:100", 201L);
+    rsTimestampMap1.put("rs3:100", 203L);
 
     table.writeRegionServerLogTimestamp(tables1, rsTimestampMap1, "root");
 
@@ -309,22 +309,22 @@ public class TestBackupSystemTable {
       HashMap<String, Long> rstm = result.get(t);
       assertNotNull(rstm);
       if (t.equals(TableName.valueOf("t3")) == false) {
-        assertEquals(rstm.get("rs1"), new Long(100L));
-        assertEquals(rstm.get("rs2"), new Long(101L));
-        assertEquals(rstm.get("rs3"), new Long(103L));
+        assertEquals(rstm.get("rs1:100"), new Long(100L));
+        assertEquals(rstm.get("rs2:100"), new Long(101L));
+        assertEquals(rstm.get("rs3:100"), new Long(103L));
       } else {
-        assertEquals(rstm.get("rs1"), new Long(200L));
-        assertEquals(rstm.get("rs2"), new Long(201L));
-        assertEquals(rstm.get("rs3"), new Long(203L));
+        assertEquals(rstm.get("rs1:100"), new Long(200L));
+        assertEquals(rstm.get("rs2:100"), new Long(201L));
+        assertEquals(rstm.get("rs3:100"), new Long(203L));
       }
     }
 
     for (TableName t : tables1) {
       HashMap<String, Long> rstm = result.get(t);
       assertNotNull(rstm);
-      assertEquals(rstm.get("rs1"), new Long(200L));
-      assertEquals(rstm.get("rs2"), new Long(201L));
-      assertEquals(rstm.get("rs3"), new Long(203L));
+      assertEquals(rstm.get("rs1:100"), new Long(200L));
+      assertEquals(rstm.get("rs2:100"), new Long(201L));
+      assertEquals(rstm.get("rs3:100"), new Long(203L));
     }
 
     cleanBackupTable();


[3/3] hbase git commit: HBASE-16827 Backup.proto refactoring (Vladimir Rodionov)

Posted by te...@apache.org.
HBASE-16827 Backup.proto refactoring (Vladimir Rodionov)


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

Branch: refs/heads/HBASE-7912
Commit: 42e56586d135dc14ec8a6860caf9bff5bd2e8c26
Parents: 33d76f1
Author: tedyu <yu...@gmail.com>
Authored: Mon Oct 24 15:56:52 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Oct 24 15:56:52 2016 -0700

----------------------------------------------------------------------
 .../hbase/protobuf/generated/BackupProtos.java  | 4051 +++++-------------
 hbase-protocol/src/main/protobuf/Backup.proto   |   72 +-
 .../hbase/backup/impl/BackupManifest.java       |  272 +-
 .../hbase/backup/impl/BackupSystemTable.java    |   11 +-
 .../hbase/backup/TestBackupSystemTable.java     |   36 +-
 5 files changed, 1239 insertions(+), 3203 deletions(-)
----------------------------------------------------------------------



[2/3] hbase git commit: HBASE-16827 Backup.proto refactoring (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/42e56586/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java
index c5220cc..93dd0da 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java
@@ -10,1884 +10,110 @@ public final class BackupProtos {
   }
   /**
    * Protobuf enum {@code hbase.pb.BackupType}
-   *
-   * <pre>
-   *enum FullTableBackupState {
-   *PRE_SNAPSHOT_TABLE = 1;
-   *SNAPSHOT_TABLES = 2;
-   *SNAPSHOT_COPY = 3;
-   *BACKUP_COMPLETE = 4;
-   *}
-   *
-   *enum IncrementalTableBackupState {
-   *PREPARE_INCREMENTAL = 1;
-   *INCREMENTAL_COPY = 2;
-   *INCR_BACKUP_COMPLETE = 3;
-   *}
-   *
-   *message SnapshotTableStateData {
-   *required TableName table = 1;
-   *required string snapshotName = 2;
-   *}
-   * </pre>
-   */
-  public enum BackupType
-      implements com.google.protobuf.ProtocolMessageEnum {
-    /**
-     * <code>FULL = 0;</code>
-     */
-    FULL(0, 0),
-    /**
-     * <code>INCREMENTAL = 1;</code>
-     */
-    INCREMENTAL(1, 1),
-    ;
-
-    /**
-     * <code>FULL = 0;</code>
-     */
-    public static final int FULL_VALUE = 0;
-    /**
-     * <code>INCREMENTAL = 1;</code>
-     */
-    public static final int INCREMENTAL_VALUE = 1;
-
-
-    public final int getNumber() { return value; }
-
-    public static BackupType valueOf(int value) {
-      switch (value) {
-        case 0: return FULL;
-        case 1: return INCREMENTAL;
-        default: return null;
-      }
-    }
-
-    public static com.google.protobuf.Internal.EnumLiteMap<BackupType>
-        internalGetValueMap() {
-      return internalValueMap;
-    }
-    private static com.google.protobuf.Internal.EnumLiteMap<BackupType>
-        internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap<BackupType>() {
-            public BackupType findValueByNumber(int number) {
-              return BackupType.valueOf(number);
-            }
-          };
-
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
-        getValueDescriptor() {
-      return getDescriptor().getValues().get(index);
-    }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
-        getDescriptorForType() {
-      return getDescriptor();
-    }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.getDescriptor().getEnumTypes().get(0);
-    }
-
-    private static final BackupType[] VALUES = values();
-
-    public static BackupType valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
-      if (desc.getType() != getDescriptor()) {
-        throw new java.lang.IllegalArgumentException(
-          "EnumValueDescriptor is not for this type.");
-      }
-      return VALUES[desc.getIndex()];
-    }
-
-    private final int index;
-    private final int value;
-
-    private BackupType(int index, int value) {
-      this.index = index;
-      this.value = value;
-    }
-
-    // @@protoc_insertion_point(enum_scope:hbase.pb.BackupType)
-  }
-
-  public interface BackupImageOrBuilder
-      extends com.google.protobuf.MessageOrBuilder {
-
-    // required string backup_id = 1;
-    /**
-     * <code>required string backup_id = 1;</code>
-     */
-    boolean hasBackupId();
-    /**
-     * <code>required string backup_id = 1;</code>
-     */
-    java.lang.String getBackupId();
-    /**
-     * <code>required string backup_id = 1;</code>
-     */
-    com.google.protobuf.ByteString
-        getBackupIdBytes();
-
-    // required .hbase.pb.BackupType backup_type = 2;
-    /**
-     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
-     */
-    boolean hasBackupType();
-    /**
-     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType getBackupType();
-
-    // required string root_dir = 3;
-    /**
-     * <code>required string root_dir = 3;</code>
-     */
-    boolean hasRootDir();
-    /**
-     * <code>required string root_dir = 3;</code>
-     */
-    java.lang.String getRootDir();
-    /**
-     * <code>required string root_dir = 3;</code>
-     */
-    com.google.protobuf.ByteString
-        getRootDirBytes();
-
-    // repeated .hbase.pb.TableName table_list = 4;
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> 
-        getTableListList();
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableList(int index);
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    int getTableListCount();
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
-        getTableListOrBuilderList();
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableListOrBuilder(
-        int index);
-
-    // required uint64 start_ts = 5;
-    /**
-     * <code>required uint64 start_ts = 5;</code>
-     */
-    boolean hasStartTs();
-    /**
-     * <code>required uint64 start_ts = 5;</code>
-     */
-    long getStartTs();
-
-    // required uint64 complete_ts = 6;
-    /**
-     * <code>required uint64 complete_ts = 6;</code>
-     */
-    boolean hasCompleteTs();
-    /**
-     * <code>required uint64 complete_ts = 6;</code>
-     */
-    long getCompleteTs();
-
-    // repeated .hbase.pb.BackupImage ancestors = 7;
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> 
-        getAncestorsList();
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage getAncestors(int index);
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    int getAncestorsCount();
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder> 
-        getAncestorsOrBuilderList();
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder getAncestorsOrBuilder(
-        int index);
-  }
-  /**
-   * Protobuf type {@code hbase.pb.BackupImage}
-   */
-  public static final class BackupImage extends
-      com.google.protobuf.GeneratedMessage
-      implements BackupImageOrBuilder {
-    // Use BackupImage.newBuilder() to construct.
-    private BackupImage(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private BackupImage(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final BackupImage defaultInstance;
-    public static BackupImage getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public BackupImage getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private BackupImage(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 10: {
-              bitField0_ |= 0x00000001;
-              backupId_ = input.readBytes();
-              break;
-            }
-            case 16: {
-              int rawValue = input.readEnum();
-              org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType value = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.valueOf(rawValue);
-              if (value == null) {
-                unknownFields.mergeVarintField(2, rawValue);
-              } else {
-                bitField0_ |= 0x00000002;
-                backupType_ = value;
-              }
-              break;
-            }
-            case 26: {
-              bitField0_ |= 0x00000004;
-              rootDir_ = input.readBytes();
-              break;
-            }
-            case 34: {
-              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
-                tableList_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName>();
-                mutable_bitField0_ |= 0x00000008;
-              }
-              tableList_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry));
-              break;
-            }
-            case 40: {
-              bitField0_ |= 0x00000008;
-              startTs_ = input.readUInt64();
-              break;
-            }
-            case 48: {
-              bitField0_ |= 0x00000010;
-              completeTs_ = input.readUInt64();
-              break;
-            }
-            case 58: {
-              if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
-                ancestors_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage>();
-                mutable_bitField0_ |= 0x00000040;
-              }
-              ancestors_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.PARSER, extensionRegistry));
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
-          tableList_ = java.util.Collections.unmodifiableList(tableList_);
-        }
-        if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
-          ancestors_ = java.util.Collections.unmodifiableList(ancestors_);
-        }
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.class, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<BackupImage> PARSER =
-        new com.google.protobuf.AbstractParser<BackupImage>() {
-      public BackupImage parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new BackupImage(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<BackupImage> getParserForType() {
-      return PARSER;
-    }
-
-    private int bitField0_;
-    // required string backup_id = 1;
-    public static final int BACKUP_ID_FIELD_NUMBER = 1;
-    private java.lang.Object backupId_;
-    /**
-     * <code>required string backup_id = 1;</code>
-     */
-    public boolean hasBackupId() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required string backup_id = 1;</code>
-     */
-    public java.lang.String getBackupId() {
-      java.lang.Object ref = backupId_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          backupId_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>required string backup_id = 1;</code>
-     */
-    public com.google.protobuf.ByteString
-        getBackupIdBytes() {
-      java.lang.Object ref = backupId_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        backupId_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // required .hbase.pb.BackupType backup_type = 2;
-    public static final int BACKUP_TYPE_FIELD_NUMBER = 2;
-    private org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType backupType_;
-    /**
-     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
-     */
-    public boolean hasBackupType() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType getBackupType() {
-      return backupType_;
-    }
-
-    // required string root_dir = 3;
-    public static final int ROOT_DIR_FIELD_NUMBER = 3;
-    private java.lang.Object rootDir_;
-    /**
-     * <code>required string root_dir = 3;</code>
-     */
-    public boolean hasRootDir() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
-    }
-    /**
-     * <code>required string root_dir = 3;</code>
-     */
-    public java.lang.String getRootDir() {
-      java.lang.Object ref = rootDir_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          rootDir_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>required string root_dir = 3;</code>
-     */
-    public com.google.protobuf.ByteString
-        getRootDirBytes() {
-      java.lang.Object ref = rootDir_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        rootDir_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // repeated .hbase.pb.TableName table_list = 4;
-    public static final int TABLE_LIST_FIELD_NUMBER = 4;
-    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> tableList_;
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> getTableListList() {
-      return tableList_;
-    }
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
-        getTableListOrBuilderList() {
-      return tableList_;
-    }
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    public int getTableListCount() {
-      return tableList_.size();
-    }
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableList(int index) {
-      return tableList_.get(index);
-    }
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableListOrBuilder(
-        int index) {
-      return tableList_.get(index);
-    }
-
-    // required uint64 start_ts = 5;
-    public static final int START_TS_FIELD_NUMBER = 5;
-    private long startTs_;
-    /**
-     * <code>required uint64 start_ts = 5;</code>
-     */
-    public boolean hasStartTs() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
-    }
-    /**
-     * <code>required uint64 start_ts = 5;</code>
-     */
-    public long getStartTs() {
-      return startTs_;
-    }
-
-    // required uint64 complete_ts = 6;
-    public static final int COMPLETE_TS_FIELD_NUMBER = 6;
-    private long completeTs_;
-    /**
-     * <code>required uint64 complete_ts = 6;</code>
-     */
-    public boolean hasCompleteTs() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
-    }
-    /**
-     * <code>required uint64 complete_ts = 6;</code>
-     */
-    public long getCompleteTs() {
-      return completeTs_;
-    }
-
-    // repeated .hbase.pb.BackupImage ancestors = 7;
-    public static final int ANCESTORS_FIELD_NUMBER = 7;
-    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> ancestors_;
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> getAncestorsList() {
-      return ancestors_;
-    }
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder> 
-        getAncestorsOrBuilderList() {
-      return ancestors_;
-    }
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    public int getAncestorsCount() {
-      return ancestors_.size();
-    }
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage getAncestors(int index) {
-      return ancestors_.get(index);
-    }
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder getAncestorsOrBuilder(
-        int index) {
-      return ancestors_.get(index);
-    }
-
-    private void initFields() {
-      backupId_ = "";
-      backupType_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL;
-      rootDir_ = "";
-      tableList_ = java.util.Collections.emptyList();
-      startTs_ = 0L;
-      completeTs_ = 0L;
-      ancestors_ = java.util.Collections.emptyList();
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      if (!hasBackupId()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasBackupType()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasRootDir()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasStartTs()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasCompleteTs()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      for (int i = 0; i < getTableListCount(); i++) {
-        if (!getTableList(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
-      for (int i = 0; i < getAncestorsCount(); i++) {
-        if (!getAncestors(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBytes(1, getBackupIdBytes());
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeEnum(2, backupType_.getNumber());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeBytes(3, getRootDirBytes());
-      }
-      for (int i = 0; i < tableList_.size(); i++) {
-        output.writeMessage(4, tableList_.get(i));
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeUInt64(5, startTs_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeUInt64(6, completeTs_);
-      }
-      for (int i = 0; i < ancestors_.size(); i++) {
-        output.writeMessage(7, ancestors_.get(i));
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(1, getBackupIdBytes());
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeEnumSize(2, backupType_.getNumber());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(3, getRootDirBytes());
-      }
-      for (int i = 0; i < tableList_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(4, tableList_.get(i));
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(5, startTs_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(6, completeTs_);
-      }
-      for (int i = 0; i < ancestors_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(7, ancestors_.get(i));
-      }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage)) {
-        return super.equals(obj);
-      }
-      org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage other = (org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage) obj;
-
-      boolean result = true;
-      result = result && (hasBackupId() == other.hasBackupId());
-      if (hasBackupId()) {
-        result = result && getBackupId()
-            .equals(other.getBackupId());
-      }
-      result = result && (hasBackupType() == other.hasBackupType());
-      if (hasBackupType()) {
-        result = result &&
-            (getBackupType() == other.getBackupType());
-      }
-      result = result && (hasRootDir() == other.hasRootDir());
-      if (hasRootDir()) {
-        result = result && getRootDir()
-            .equals(other.getRootDir());
-      }
-      result = result && getTableListList()
-          .equals(other.getTableListList());
-      result = result && (hasStartTs() == other.hasStartTs());
-      if (hasStartTs()) {
-        result = result && (getStartTs()
-            == other.getStartTs());
-      }
-      result = result && (hasCompleteTs() == other.hasCompleteTs());
-      if (hasCompleteTs()) {
-        result = result && (getCompleteTs()
-            == other.getCompleteTs());
-      }
-      result = result && getAncestorsList()
-          .equals(other.getAncestorsList());
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
-
-    private int memoizedHashCode = 0;
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasBackupId()) {
-        hash = (37 * hash) + BACKUP_ID_FIELD_NUMBER;
-        hash = (53 * hash) + getBackupId().hashCode();
-      }
-      if (hasBackupType()) {
-        hash = (37 * hash) + BACKUP_TYPE_FIELD_NUMBER;
-        hash = (53 * hash) + hashEnum(getBackupType());
-      }
-      if (hasRootDir()) {
-        hash = (37 * hash) + ROOT_DIR_FIELD_NUMBER;
-        hash = (53 * hash) + getRootDir().hashCode();
-      }
-      if (getTableListCount() > 0) {
-        hash = (37 * hash) + TABLE_LIST_FIELD_NUMBER;
-        hash = (53 * hash) + getTableListList().hashCode();
-      }
-      if (hasStartTs()) {
-        hash = (37 * hash) + START_TS_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getStartTs());
-      }
-      if (hasCompleteTs()) {
-        hash = (37 * hash) + COMPLETE_TS_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getCompleteTs());
-      }
-      if (getAncestorsCount() > 0) {
-        hash = (37 * hash) + ANCESTORS_FIELD_NUMBER;
-        hash = (53 * hash) + getAncestorsList().hashCode();
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-
-    public static Builder newBuilder() { return Builder.create(); }
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage prototype) {
-      return newBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() { return newBuilder(this); }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code hbase.pb.BackupImage}
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.class, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder.class);
-      }
-
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          getTableListFieldBuilder();
-          getAncestorsFieldBuilder();
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
-
-      public Builder clear() {
-        super.clear();
-        backupId_ = "";
-        bitField0_ = (bitField0_ & ~0x00000001);
-        backupType_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL;
-        bitField0_ = (bitField0_ & ~0x00000002);
-        rootDir_ = "";
-        bitField0_ = (bitField0_ & ~0x00000004);
-        if (tableListBuilder_ == null) {
-          tableList_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000008);
-        } else {
-          tableListBuilder_.clear();
-        }
-        startTs_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000010);
-        completeTs_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000020);
-        if (ancestorsBuilder_ == null) {
-          ancestors_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000040);
-        } else {
-          ancestorsBuilder_.clear();
-        }
-        return this;
-      }
-
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_descriptor;
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.getDefaultInstance();
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage build() {
-        org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage result = new org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.backupId_ = backupId_;
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        result.backupType_ = backupType_;
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000004;
-        }
-        result.rootDir_ = rootDir_;
-        if (tableListBuilder_ == null) {
-          if (((bitField0_ & 0x00000008) == 0x00000008)) {
-            tableList_ = java.util.Collections.unmodifiableList(tableList_);
-            bitField0_ = (bitField0_ & ~0x00000008);
-          }
-          result.tableList_ = tableList_;
-        } else {
-          result.tableList_ = tableListBuilder_.build();
-        }
-        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
-          to_bitField0_ |= 0x00000008;
-        }
-        result.startTs_ = startTs_;
-        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
-          to_bitField0_ |= 0x00000010;
-        }
-        result.completeTs_ = completeTs_;
-        if (ancestorsBuilder_ == null) {
-          if (((bitField0_ & 0x00000040) == 0x00000040)) {
-            ancestors_ = java.util.Collections.unmodifiableList(ancestors_);
-            bitField0_ = (bitField0_ & ~0x00000040);
-          }
-          result.ancestors_ = ancestors_;
-        } else {
-          result.ancestors_ = ancestorsBuilder_.build();
-        }
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.getDefaultInstance()) return this;
-        if (other.hasBackupId()) {
-          bitField0_ |= 0x00000001;
-          backupId_ = other.backupId_;
-          onChanged();
-        }
-        if (other.hasBackupType()) {
-          setBackupType(other.getBackupType());
-        }
-        if (other.hasRootDir()) {
-          bitField0_ |= 0x00000004;
-          rootDir_ = other.rootDir_;
-          onChanged();
-        }
-        if (tableListBuilder_ == null) {
-          if (!other.tableList_.isEmpty()) {
-            if (tableList_.isEmpty()) {
-              tableList_ = other.tableList_;
-              bitField0_ = (bitField0_ & ~0x00000008);
-            } else {
-              ensureTableListIsMutable();
-              tableList_.addAll(other.tableList_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.tableList_.isEmpty()) {
-            if (tableListBuilder_.isEmpty()) {
-              tableListBuilder_.dispose();
-              tableListBuilder_ = null;
-              tableList_ = other.tableList_;
-              bitField0_ = (bitField0_ & ~0x00000008);
-              tableListBuilder_ = 
-                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
-                   getTableListFieldBuilder() : null;
-            } else {
-              tableListBuilder_.addAllMessages(other.tableList_);
-            }
-          }
-        }
-        if (other.hasStartTs()) {
-          setStartTs(other.getStartTs());
-        }
-        if (other.hasCompleteTs()) {
-          setCompleteTs(other.getCompleteTs());
-        }
-        if (ancestorsBuilder_ == null) {
-          if (!other.ancestors_.isEmpty()) {
-            if (ancestors_.isEmpty()) {
-              ancestors_ = other.ancestors_;
-              bitField0_ = (bitField0_ & ~0x00000040);
-            } else {
-              ensureAncestorsIsMutable();
-              ancestors_.addAll(other.ancestors_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.ancestors_.isEmpty()) {
-            if (ancestorsBuilder_.isEmpty()) {
-              ancestorsBuilder_.dispose();
-              ancestorsBuilder_ = null;
-              ancestors_ = other.ancestors_;
-              bitField0_ = (bitField0_ & ~0x00000040);
-              ancestorsBuilder_ = 
-                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
-                   getAncestorsFieldBuilder() : null;
-            } else {
-              ancestorsBuilder_.addAllMessages(other.ancestors_);
-            }
-          }
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (!hasBackupId()) {
-          
-          return false;
-        }
-        if (!hasBackupType()) {
-          
-          return false;
-        }
-        if (!hasRootDir()) {
-          
-          return false;
-        }
-        if (!hasStartTs()) {
-          
-          return false;
-        }
-        if (!hasCompleteTs()) {
-          
-          return false;
-        }
-        for (int i = 0; i < getTableListCount(); i++) {
-          if (!getTableList(i).isInitialized()) {
-            
-            return false;
-          }
-        }
-        for (int i = 0; i < getAncestorsCount(); i++) {
-          if (!getAncestors(i).isInitialized()) {
-            
-            return false;
-          }
-        }
-        return true;
-      }
-
-      public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      // required string backup_id = 1;
-      private java.lang.Object backupId_ = "";
-      /**
-       * <code>required string backup_id = 1;</code>
-       */
-      public boolean hasBackupId() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>required string backup_id = 1;</code>
-       */
-      public java.lang.String getBackupId() {
-        java.lang.Object ref = backupId_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          backupId_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>required string backup_id = 1;</code>
-       */
-      public com.google.protobuf.ByteString
-          getBackupIdBytes() {
-        java.lang.Object ref = backupId_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          backupId_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>required string backup_id = 1;</code>
-       */
-      public Builder setBackupId(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-        backupId_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string backup_id = 1;</code>
-       */
-      public Builder clearBackupId() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        backupId_ = getDefaultInstance().getBackupId();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string backup_id = 1;</code>
-       */
-      public Builder setBackupIdBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-        backupId_ = value;
-        onChanged();
-        return this;
-      }
-
-      // required .hbase.pb.BackupType backup_type = 2;
-      private org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType backupType_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL;
-      /**
-       * <code>required .hbase.pb.BackupType backup_type = 2;</code>
-       */
-      public boolean hasBackupType() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>required .hbase.pb.BackupType backup_type = 2;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType getBackupType() {
-        return backupType_;
-      }
-      /**
-       * <code>required .hbase.pb.BackupType backup_type = 2;</code>
-       */
-      public Builder setBackupType(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType value) {
-        if (value == null) {
-          throw new NullPointerException();
-        }
-        bitField0_ |= 0x00000002;
-        backupType_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required .hbase.pb.BackupType backup_type = 2;</code>
-       */
-      public Builder clearBackupType() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        backupType_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL;
-        onChanged();
-        return this;
-      }
-
-      // required string root_dir = 3;
-      private java.lang.Object rootDir_ = "";
-      /**
-       * <code>required string root_dir = 3;</code>
-       */
-      public boolean hasRootDir() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
-      }
-      /**
-       * <code>required string root_dir = 3;</code>
-       */
-      public java.lang.String getRootDir() {
-        java.lang.Object ref = rootDir_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          rootDir_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>required string root_dir = 3;</code>
-       */
-      public com.google.protobuf.ByteString
-          getRootDirBytes() {
-        java.lang.Object ref = rootDir_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          rootDir_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>required string root_dir = 3;</code>
-       */
-      public Builder setRootDir(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000004;
-        rootDir_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string root_dir = 3;</code>
-       */
-      public Builder clearRootDir() {
-        bitField0_ = (bitField0_ & ~0x00000004);
-        rootDir_ = getDefaultInstance().getRootDir();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string root_dir = 3;</code>
-       */
-      public Builder setRootDirBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000004;
-        rootDir_ = value;
-        onChanged();
-        return this;
-      }
-
-      // repeated .hbase.pb.TableName table_list = 4;
-      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> tableList_ =
-        java.util.Collections.emptyList();
-      private void ensureTableListIsMutable() {
-        if (!((bitField0_ & 0x00000008) == 0x00000008)) {
-          tableList_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName>(tableList_);
-          bitField0_ |= 0x00000008;
-         }
-      }
-
-      private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableListBuilder_;
-
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> getTableListList() {
-        if (tableListBuilder_ == null) {
-          return java.util.Collections.unmodifiableList(tableList_);
-        } else {
-          return tableListBuilder_.getMessageList();
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public int getTableListCount() {
-        if (tableListBuilder_ == null) {
-          return tableList_.size();
-        } else {
-          return tableListBuilder_.getCount();
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableList(int index) {
-        if (tableListBuilder_ == null) {
-          return tableList_.get(index);
-        } else {
-          return tableListBuilder_.getMessage(index);
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public Builder setTableList(
-          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
-        if (tableListBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureTableListIsMutable();
-          tableList_.set(index, value);
-          onChanged();
-        } else {
-          tableListBuilder_.setMessage(index, value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public Builder setTableList(
-          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
-        if (tableListBuilder_ == null) {
-          ensureTableListIsMutable();
-          tableList_.set(index, builderForValue.build());
-          onChanged();
-        } else {
-          tableListBuilder_.setMessage(index, builderForValue.build());
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public Builder addTableList(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
-        if (tableListBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureTableListIsMutable();
-          tableList_.add(value);
-          onChanged();
-        } else {
-          tableListBuilder_.addMessage(value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public Builder addTableList(
-          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
-        if (tableListBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureTableListIsMutable();
-          tableList_.add(index, value);
-          onChanged();
-        } else {
-          tableListBuilder_.addMessage(index, value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public Builder addTableList(
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
-        if (tableListBuilder_ == null) {
-          ensureTableListIsMutable();
-          tableList_.add(builderForValue.build());
-          onChanged();
-        } else {
-          tableListBuilder_.addMessage(builderForValue.build());
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public Builder addTableList(
-          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
-        if (tableListBuilder_ == null) {
-          ensureTableListIsMutable();
-          tableList_.add(index, builderForValue.build());
-          onChanged();
-        } else {
-          tableListBuilder_.addMessage(index, builderForValue.build());
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public Builder addAllTableList(
-          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> values) {
-        if (tableListBuilder_ == null) {
-          ensureTableListIsMutable();
-          super.addAll(values, tableList_);
-          onChanged();
-        } else {
-          tableListBuilder_.addAllMessages(values);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public Builder clearTableList() {
-        if (tableListBuilder_ == null) {
-          tableList_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000008);
-          onChanged();
-        } else {
-          tableListBuilder_.clear();
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public Builder removeTableList(int index) {
-        if (tableListBuilder_ == null) {
-          ensureTableListIsMutable();
-          tableList_.remove(index);
-          onChanged();
-        } else {
-          tableListBuilder_.remove(index);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableListBuilder(
-          int index) {
-        return getTableListFieldBuilder().getBuilder(index);
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableListOrBuilder(
-          int index) {
-        if (tableListBuilder_ == null) {
-          return tableList_.get(index);  } else {
-          return tableListBuilder_.getMessageOrBuilder(index);
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
-           getTableListOrBuilderList() {
-        if (tableListBuilder_ != null) {
-          return tableListBuilder_.getMessageOrBuilderList();
-        } else {
-          return java.util.Collections.unmodifiableList(tableList_);
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder addTableListBuilder() {
-        return getTableListFieldBuilder().addBuilder(
-            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance());
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder addTableListBuilder(
-          int index) {
-        return getTableListFieldBuilder().addBuilder(
-            index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance());
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-       */
-      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder> 
-           getTableListBuilderList() {
-        return getTableListFieldBuilder().getBuilderList();
-      }
-      private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
-          getTableListFieldBuilder() {
-        if (tableListBuilder_ == null) {
-          tableListBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
-                  tableList_,
-                  ((bitField0_ & 0x00000008) == 0x00000008),
-                  getParentForChildren(),
-                  isClean());
-          tableList_ = null;
-        }
-        return tableListBuilder_;
-      }
-
-      // required uint64 start_ts = 5;
-      private long startTs_ ;
-      /**
-       * <code>required uint64 start_ts = 5;</code>
-       */
-      public boolean hasStartTs() {
-        return ((bitField0_ & 0x00000010) == 0x00000010);
-      }
-      /**
-       * <code>required uint64 start_ts = 5;</code>
-       */
-      public long getStartTs() {
-        return startTs_;
-      }
-      /**
-       * <code>required uint64 start_ts = 5;</code>
-       */
-      public Builder setStartTs(long value) {
-        bitField0_ |= 0x00000010;
-        startTs_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required uint64 start_ts = 5;</code>
-       */
-      public Builder clearStartTs() {
-        bitField0_ = (bitField0_ & ~0x00000010);
-        startTs_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // required uint64 complete_ts = 6;
-      private long completeTs_ ;
-      /**
-       * <code>required uint64 complete_ts = 6;</code>
-       */
-      public boolean hasCompleteTs() {
-        return ((bitField0_ & 0x00000020) == 0x00000020);
-      }
-      /**
-       * <code>required uint64 complete_ts = 6;</code>
-       */
-      public long getCompleteTs() {
-        return completeTs_;
-      }
-      /**
-       * <code>required uint64 complete_ts = 6;</code>
-       */
-      public Builder setCompleteTs(long value) {
-        bitField0_ |= 0x00000020;
-        completeTs_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required uint64 complete_ts = 6;</code>
-       */
-      public Builder clearCompleteTs() {
-        bitField0_ = (bitField0_ & ~0x00000020);
-        completeTs_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // repeated .hbase.pb.BackupImage ancestors = 7;
-      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> ancestors_ =
-        java.util.Collections.emptyList();
-      private void ensureAncestorsIsMutable() {
-        if (!((bitField0_ & 0x00000040) == 0x00000040)) {
-          ancestors_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage>(ancestors_);
-          bitField0_ |= 0x00000040;
-         }
-      }
-
-      private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder> ancestorsBuilder_;
-
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> getAncestorsList() {
-        if (ancestorsBuilder_ == null) {
-          return java.util.Collections.unmodifiableList(ancestors_);
-        } else {
-          return ancestorsBuilder_.getMessageList();
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public int getAncestorsCount() {
-        if (ancestorsBuilder_ == null) {
-          return ancestors_.size();
-        } else {
-          return ancestorsBuilder_.getCount();
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage getAncestors(int index) {
-        if (ancestorsBuilder_ == null) {
-          return ancestors_.get(index);
-        } else {
-          return ancestorsBuilder_.getMessage(index);
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public Builder setAncestors(
-          int index, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage value) {
-        if (ancestorsBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureAncestorsIsMutable();
-          ancestors_.set(index, value);
-          onChanged();
-        } else {
-          ancestorsBuilder_.setMessage(index, value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public Builder setAncestors(
-          int index, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder builderForValue) {
-        if (ancestorsBuilder_ == null) {
-          ensureAncestorsIsMutable();
-          ancestors_.set(index, builderForValue.build());
-          onChanged();
-        } else {
-          ancestorsBuilder_.setMessage(index, builderForValue.build());
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public Builder addAncestors(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage value) {
-        if (ancestorsBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureAncestorsIsMutable();
-          ancestors_.add(value);
-          onChanged();
-        } else {
-          ancestorsBuilder_.addMessage(value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public Builder addAncestors(
-          int index, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage value) {
-        if (ancestorsBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureAncestorsIsMutable();
-          ancestors_.add(index, value);
-          onChanged();
-        } else {
-          ancestorsBuilder_.addMessage(index, value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public Builder addAncestors(
-          org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder builderForValue) {
-        if (ancestorsBuilder_ == null) {
-          ensureAncestorsIsMutable();
-          ancestors_.add(builderForValue.build());
-          onChanged();
-        } else {
-          ancestorsBuilder_.addMessage(builderForValue.build());
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public Builder addAncestors(
-          int index, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder builderForValue) {
-        if (ancestorsBuilder_ == null) {
-          ensureAncestorsIsMutable();
-          ancestors_.add(index, builderForValue.build());
-          onChanged();
-        } else {
-          ancestorsBuilder_.addMessage(index, builderForValue.build());
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public Builder addAllAncestors(
-          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> values) {
-        if (ancestorsBuilder_ == null) {
-          ensureAncestorsIsMutable();
-          super.addAll(values, ancestors_);
-          onChanged();
-        } else {
-          ancestorsBuilder_.addAllMessages(values);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public Builder clearAncestors() {
-        if (ancestorsBuilder_ == null) {
-          ancestors_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000040);
-          onChanged();
-        } else {
-          ancestorsBuilder_.clear();
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public Builder removeAncestors(int index) {
-        if (ancestorsBuilder_ == null) {
-          ensureAncestorsIsMutable();
-          ancestors_.remove(index);
-          onChanged();
-        } else {
-          ancestorsBuilder_.remove(index);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder getAncestorsBuilder(
-          int index) {
-        return getAncestorsFieldBuilder().getBuilder(index);
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder getAncestorsOrBuilder(
-          int index) {
-        if (ancestorsBuilder_ == null) {
-          return ancestors_.get(index);  } else {
-          return ancestorsBuilder_.getMessageOrBuilder(index);
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder> 
-           getAncestorsOrBuilderList() {
-        if (ancestorsBuilder_ != null) {
-          return ancestorsBuilder_.getMessageOrBuilderList();
-        } else {
-          return java.util.Collections.unmodifiableList(ancestors_);
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder addAncestorsBuilder() {
-        return getAncestorsFieldBuilder().addBuilder(
-            org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.getDefaultInstance());
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder addAncestorsBuilder(
-          int index) {
-        return getAncestorsFieldBuilder().addBuilder(
-            index, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.getDefaultInstance());
-      }
-      /**
-       * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-       */
-      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder> 
-           getAncestorsBuilderList() {
-        return getAncestorsFieldBuilder().getBuilderList();
-      }
-      private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder> 
-          getAncestorsFieldBuilder() {
-        if (ancestorsBuilder_ == null) {
-          ancestorsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
-              org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder>(
-                  ancestors_,
-                  ((bitField0_ & 0x00000040) == 0x00000040),
-                  getParentForChildren(),
-                  isClean());
-          ancestors_ = null;
-        }
-        return ancestorsBuilder_;
+   */
+  public enum BackupType
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>FULL = 0;</code>
+     */
+    FULL(0, 0),
+    /**
+     * <code>INCREMENTAL = 1;</code>
+     */
+    INCREMENTAL(1, 1),
+    ;
+
+    /**
+     * <code>FULL = 0;</code>
+     */
+    public static final int FULL_VALUE = 0;
+    /**
+     * <code>INCREMENTAL = 1;</code>
+     */
+    public static final int INCREMENTAL_VALUE = 1;
+
+
+    public final int getNumber() { return value; }
+
+    public static BackupType valueOf(int value) {
+      switch (value) {
+        case 0: return FULL;
+        case 1: return INCREMENTAL;
+        default: return null;
       }
+    }
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.BackupImage)
+    public static com.google.protobuf.Internal.EnumLiteMap<BackupType>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<BackupType>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<BackupType>() {
+            public BackupType findValueByNumber(int number) {
+              return BackupType.valueOf(number);
+            }
+          };
+
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(index);
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.getDescriptor().getEnumTypes().get(0);
     }
 
-    static {
-      defaultInstance = new BackupImage(true);
-      defaultInstance.initFields();
+    private static final BackupType[] VALUES = values();
+
+    public static BackupType valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.BackupImage)
+    private final int index;
+    private final int value;
+
+    private BackupType(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:hbase.pb.BackupType)
   }
 
   public interface ServerTimestampOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // required string server = 1;
+    // optional .hbase.pb.ServerName server = 1;
     /**
-     * <code>required string server = 1;</code>
+     * <code>optional .hbase.pb.ServerName server = 1;</code>
      */
     boolean hasServer();
     /**
-     * <code>required string server = 1;</code>
+     * <code>optional .hbase.pb.ServerName server = 1;</code>
      */
-    java.lang.String getServer();
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServer();
     /**
-     * <code>required string server = 1;</code>
+     * <code>optional .hbase.pb.ServerName server = 1;</code>
      */
-    com.google.protobuf.ByteString
-        getServerBytes();
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder();
 
-    // required uint64 timestamp = 2;
+    // optional uint64 timestamp = 2;
     /**
-     * <code>required uint64 timestamp = 2;</code>
+     * <code>optional uint64 timestamp = 2;</code>
      */
     boolean hasTimestamp();
     /**
-     * <code>required uint64 timestamp = 2;</code>
+     * <code>optional uint64 timestamp = 2;</code>
      */
     long getTimestamp();
   }
@@ -1943,8 +169,16 @@ public final class BackupProtos {
               break;
             }
             case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = server_.toBuilder();
+              }
+              server_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(server_);
+                server_ = subBuilder.buildPartial();
+              }
               bitField0_ |= 0x00000001;
-              server_ = input.readBytes();
               break;
             }
             case 16: {
@@ -1992,67 +226,46 @@ public final class BackupProtos {
     }
 
     private int bitField0_;
-    // required string server = 1;
+    // optional .hbase.pb.ServerName server = 1;
     public static final int SERVER_FIELD_NUMBER = 1;
-    private java.lang.Object server_;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName server_;
     /**
-     * <code>required string server = 1;</code>
+     * <code>optional .hbase.pb.ServerName server = 1;</code>
      */
     public boolean hasServer() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>required string server = 1;</code>
+     * <code>optional .hbase.pb.ServerName server = 1;</code>
      */
-    public java.lang.String getServer() {
-      java.lang.Object ref = server_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          server_ = s;
-        }
-        return s;
-      }
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServer() {
+      return server_;
     }
     /**
-     * <code>required string server = 1;</code>
+     * <code>optional .hbase.pb.ServerName server = 1;</code>
      */
-    public com.google.protobuf.ByteString
-        getServerBytes() {
-      java.lang.Object ref = server_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        server_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder() {
+      return server_;
     }
 
-    // required uint64 timestamp = 2;
+    // optional uint64 timestamp = 2;
     public static final int TIMESTAMP_FIELD_NUMBER = 2;
     private long timestamp_;
     /**
-     * <code>required uint64 timestamp = 2;</code>
+     * <code>optional uint64 timestamp = 2;</code>
      */
     public boolean hasTimestamp() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>required uint64 timestamp = 2;</code>
+     * <code>optional uint64 timestamp = 2;</code>
      */
     public long getTimestamp() {
       return timestamp_;
     }
 
     private void initFields() {
-      server_ = "";
+      server_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
       timestamp_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
@@ -2060,13 +273,11 @@ public final class BackupProtos {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
-      if (!hasServer()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasTimestamp()) {
-        memoizedIsInitialized = 0;
-        return false;
+      if (hasServer()) {
+        if (!getServer().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
       }
       memoizedIsInitialized = 1;
       return true;
@@ -2076,7 +287,7 @@ public final class BackupProtos {
                         throws java.io.IOException {
       getSerializedSize();
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBytes(1, getServerBytes());
+        output.writeMessage(1, server_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeUInt64(2, timestamp_);
@@ -2092,7 +303,7 @@ public final class BackupProtos {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(1, getServerBytes());
+          .computeMessageSize(1, server_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
@@ -2253,6 +464,7 @@ public final class BackupProtos {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getServerFieldBuilder();
         }
       }
       private static Builder create() {
@@ -2261,7 +473,11 @@ public final class BackupProtos {
 
       public Builder clear() {
         super.clear();
-        server_ = "";
+        if (serverBuilder_ == null) {
+          server_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
+        } else {
+          serverBuilder_.clear();
+        }
         bitField0_ = (bitField0_ & ~0x00000001);
         timestamp_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000002);
@@ -2296,7 +512,11 @@ public final class BackupProtos {
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        result.server_ = server_;
+        if (serverBuilder_ == null) {
+          result.server_ = server_;
+        } else {
+          result.server_ = serverBuilder_.build();
+        }
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
@@ -2318,9 +538,7 @@ public final class BackupProtos {
       public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp other) {
         if (other == org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp.getDefaultInstance()) return this;
         if (other.hasServer()) {
-          bitField0_ |= 0x00000001;
-          server_ = other.server_;
-          onChanged();
+          mergeServer(other.getServer());
         }
         if (other.hasTimestamp()) {
           setTimestamp(other.getTimestamp());
@@ -2330,13 +548,11 @@ public final class BackupProtos {
       }
 
       public final boolean isInitialized() {
-        if (!hasServer()) {
-          
-          return false;
-        }
-        if (!hasTimestamp()) {
-          
-          return false;
+        if (hasServer()) {
+          if (!getServer().isInitialized()) {
+            
+            return false;
+          }
         }
         return true;
       }
@@ -2360,96 +576,139 @@ public final class BackupProtos {
       }
       private int bitField0_;
 
-      // required string server = 1;
-      private java.lang.Object server_ = "";
+      // optional .hbase.pb.ServerName server = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName server_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverBuilder_;
       /**
-       * <code>required string server = 1;</code>
+       * <code>optional .hbase.pb.ServerName server = 1;</code>
        */
       public boolean hasServer() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>required string server = 1;</code>
+       * <code>optional .hbase.pb.ServerName server = 1;</code>
        */
-      public java.lang.String getServer() {
-        java.lang.Object ref = server_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          server_ = s;
-          return s;
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServer() {
+        if (serverBuilder_ == null) {
+          return server_;
         } else {
-          return (java.lang.String) ref;
+          return serverBuilder_.getMessage();
         }
       }
       /**
-       * <code>required string server = 1;</code>
+       * <code>optional .hbase.pb.ServerName server = 1;</code>
        */
-      public com.google.protobuf.ByteString
-          getServerBytes() {
-        java.lang.Object ref = server_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          server_ = b;
-          return b;
+      public Builder setServer(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) {
+        if (serverBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          server_ = value;
+          onChanged();
         } else {
-          return (com.google.protobuf.ByteString) ref;
+          serverBuilder_.setMessage(value);
         }
+        bitField0_ |= 0x00000001;
+        return this;
       }
       /**
-       * <code>required string server = 1;</code>
+       * <code>optional .hbase.pb.ServerName server = 1;</code>
        */
       public Builder setServer(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-        server_ = value;
-        onChanged();
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) {
+        if (serverBuilder_ == null) {
+          server_ = builderForValue.build();
+          onChanged();
+        } else {
+          serverBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.ServerName server = 1;</code>
+       */
+      public Builder mergeServer(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) {
+        if (serverBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              server_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()) {
+            server_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder(server_).mergeFrom(value).buildPartial();
+          } else {
+            server_ = value;
+          }
+          onChanged();
+        } else {
+          serverBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.ServerName server = 1;</code>
+       */
+      public Builder clearServer() {
+        if (serverBuilder_ == null) {
+          server_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance();
+          onChanged();
+        } else {
+          serverBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
       /**
-       * <code>required string server = 1;</code>
+       * <code>optional .hbase.pb.ServerName server = 1;</code>
        */
-      public Builder clearServer() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        server_ = getDefaultInstance().getServer();
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder getServerBuilder() {
+        bitField0_ |= 0x00000001;
         onChanged();
-        return this;
+        return getServerFieldBuilder().getBuilder();
       }
       /**
-       * <code>required string server = 1;</code>
+       * <code>optional .hbase.pb.ServerName server = 1;</code>
        */
-      public Builder setServerBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-        server_ = value;
-        onChanged();
-        return this;
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder() {
+        if (serverBuilder_ != null) {
+          return serverBuilder_.getMessageOrBuilder();
+        } else {
+          return server_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.ServerName server = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
+          getServerFieldBuilder() {
+        if (serverBuilder_ == null) {
+          serverBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
+                  server_,
+                  getParentForChildren(),
+                  isClean());
+          server_ = null;
+        }
+        return serverBuilder_;
       }
 
-      // required uint64 timestamp = 2;
+      // optional uint64 timestamp = 2;
       private long timestamp_ ;
       /**
-       * <code>required uint64 timestamp = 2;</code>
+       * <code>optional uint64 timestamp = 2;</code>
        */
       public boolean hasTimestamp() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>required uint64 timestamp = 2;</code>
+       * <code>optional uint64 timestamp = 2;</code>
        */
       public long getTimestamp() {
         return timestamp_;
       }
       /**
-       * <code>required uint64 timestamp = 2;</code>
+       * <code>optional uint64 timestamp = 2;</code>
        */
       public Builder setTimestamp(long value) {
         bitField0_ |= 0x00000002;
@@ -2458,7 +717,7 @@ public final class BackupProtos {
         return this;
       }
       /**
-       * <code>required uint64 timestamp = 2;</code>
+       * <code>optional uint64 timestamp = 2;</code>
        */
       public Builder clearTimestamp() {
         bitField0_ = (bitField0_ & ~0x00000002);
@@ -2481,17 +740,17 @@ public final class BackupProtos {
   public interface TableServerTimestampOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // required .hbase.pb.TableName table = 1;
+    // optional .hbase.pb.TableName table = 1;
     /**
-     * <code>required .hbase.pb.TableName table = 1;</code>
+     * <code>optional .hbase.pb.TableName table = 1;</code>
      */
     boolean hasTable();
     /**
-     * <code>required .hbase.pb.TableName table = 1;</code>
+     * <code>optional .hbase.pb.TableName table = 1;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTable();
     /**
-     * <code>required .hbase.pb.TableName table = 1;</code>
+     * <code>optional .hbase.pb.TableName table = 1;</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableOrBuilder();
 
@@ -2635,23 +894,23 @@ public final class BackupProtos {
     }
 
     private int bitField0_;
-    // required .hbase.pb.TableName table = 1;
+    // optional .hbase.pb.TableName table = 1;
     public static final int TABLE_FIELD_NUMBER = 1;
     private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName table_;
     /**
-     * <code>required .hbase.pb.TableName table = 1;</code>
+     * <code>optional .hbase.pb.TableName table = 1;</code>
      */
     public boolean hasTable() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>required .hbase.pb.TableName table = 1;</code>
+     * <code>optional .hbase.pb.TableName table = 1;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTable() {
       return table_;
     }
     /**
-     * <code>required .hbase.pb.TableName table = 1;</code>
+     * <code>optional .hbase.pb.TableName table = 1;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableOrBuilder() {
       return table_;
@@ -2702,13 +961,11 @@ public final class BackupProtos {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
-      if (!hasTable()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!getTable().isInitialized()) {
-        memoizedIsInitialized = 0;
-        return false;
+      if (hasTable()) {
+        if (!getTable().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
       }
       for (int i = 0; i < getServerTimestampCount(); i++) {
         if (!getServerTimestamp(i).isInitialized()) {
@@ -3015,13 +1272,11 @@ public final class BackupProtos {
       }
 
       public final boolean isInitialized() {
-        if (!hasTable())

<TRUNCATED>