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/03/23 23:40:01 UTC

hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 - renaming BackupUtility

Repository: hbase
Updated Branches:
  refs/heads/HBASE-7912 ab491d4a2 -> 524edfdbd


HBASE-14030 HBase Backup/Restore Phase 1 - renaming BackupUtility


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

Branch: refs/heads/HBASE-7912
Commit: 524edfdbd38c720f6db773652450afe29bb6e5db
Parents: ab491d4
Author: tedyu <yu...@gmail.com>
Authored: Tue Mar 22 10:08:47 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Mar 22 10:08:47 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/backup/BackupClientUtil.java   | 165 ++++++++++++++++++
 .../hadoop/hbase/backup/BackupUtility.java      | 169 -------------------
 .../hbase/backup/impl/BackupClientImpl.java     |   4 +-
 .../hadoop/hbase/backup/impl/BackupHandler.java |   4 +-
 .../hadoop/hbase/backup/impl/BackupUtil.java    |  20 +--
 .../backup/impl/IncrementalBackupManager.java   |  10 +-
 .../hbase/backup/impl/RestoreClientImpl.java    |   4 +-
 7 files changed, 186 insertions(+), 190 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/524edfdb/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupClientUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupClientUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupClientUtil.java
new file mode 100644
index 0000000..072adaf
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupClientUtil.java
@@ -0,0 +1,165 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * A collection of methods used by multiple classes to backup HBase tables.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class BackupClientUtil {
+  protected static final Log LOG = LogFactory.getLog(BackupClientUtil.class);
+  public static final String LOGNAME_SEPARATOR = ".";
+
+  private BackupClientUtil(){
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  /**
+   * Check whether the backup path exist
+   * @param backupStr backup
+   * @param conf configuration
+   * @return Yes if path exists
+   * @throws IOException exception
+   */
+  public static boolean checkPathExist(String backupStr, Configuration conf)
+    throws IOException {
+    boolean isExist = false;
+    Path backupPath = new Path(backupStr);
+    FileSystem fileSys = backupPath.getFileSystem(conf);
+    String targetFsScheme = fileSys.getUri().getScheme();
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Schema of given url: " + backupStr + " is: " + targetFsScheme);
+    }
+    if (fileSys.exists(backupPath)) {
+      isExist = true;
+    }
+    return isExist;
+  }
+
+  // check target path first, confirm it doesn't exist before backup
+  public static void checkTargetDir(String backupRootPath, Configuration conf) throws IOException {
+    boolean targetExists = false;
+    try {
+      targetExists = checkPathExist(backupRootPath, conf);
+    } catch (IOException e) {
+      String expMsg = e.getMessage();
+      String newMsg = null;
+      if (expMsg.contains("No FileSystem for scheme")) {
+        newMsg =
+            "Unsupported filesystem scheme found in the backup target url. Error Message: "
+                + newMsg;
+        LOG.error(newMsg);
+        throw new IOException(newMsg);
+      } else {
+        throw e;
+      }
+    } catch (RuntimeException e) {
+      LOG.error(e.getMessage());
+      throw e;
+    }
+
+    if (targetExists) {
+      LOG.info("Using existing backup root dir: " + backupRootPath);
+    } else {
+      LOG.info("Backup root dir " + backupRootPath + " does not exist. Will be created.");
+    }
+  }
+
+  /**
+   * Get the min value for all the Values a map.
+   * @param map map
+   * @return the min value
+   */
+  public static <T> Long getMinValue(HashMap<T, Long> map) {
+    Long minTimestamp = null;
+    if (map != null) {
+      ArrayList<Long> timestampList = new ArrayList<Long>(map.values());
+      Collections.sort(timestampList);
+      // The min among all the RS log timestamps will be kept in hbase:backup table.
+      minTimestamp = timestampList.get(0);
+    }
+    return minTimestamp;
+  }
+
+  /**
+   * TODO: verify the code
+   * @param p path
+   * @return host name
+   * @throws IOException exception
+   */
+  public static String parseHostFromOldLog(Path p) throws IOException {
+    String n = p.getName();
+    int idx = n.lastIndexOf(LOGNAME_SEPARATOR);
+    String s = URLDecoder.decode(n.substring(0, idx), "UTF8");
+    return ServerName.parseHostname(s) + ":" + ServerName.parsePort(s);
+  }
+
+  /**
+   * Given the log file, parse the timestamp from the file name. The timestamp is the last number.
+   * @param p a path to the log file
+   * @return the timestamp
+   * @throws IOException exception
+   */
+  public static Long getCreationTime(Path p) throws IOException {
+    int idx = p.getName().lastIndexOf(LOGNAME_SEPARATOR);
+    if (idx < 0) {
+      throw new IOException("Cannot parse timestamp from path " + p);
+    }
+    String ts = p.getName().substring(idx + 1);
+    return Long.parseLong(ts);
+  }
+
+  public static List<String> getFiles(FileSystem fs, Path rootDir, List<String> files,
+    PathFilter filter) throws FileNotFoundException, IOException {
+    RemoteIterator<LocatedFileStatus> it = fs.listFiles(rootDir, true);
+
+    while (it.hasNext()) {
+      LocatedFileStatus lfs = it.next();
+      if (lfs.isDirectory()) {
+        continue;
+      }
+      // apply filter
+      if (filter.accept(lfs.getPath())) {
+        files.add(lfs.getPath().toString());
+      }
+    }
+    return files;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/524edfdb/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupUtility.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupUtility.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupUtility.java
deleted file mode 100644
index 3e96f66..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupUtility.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.URLDecoder;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.TreeMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-/**
- * A collection of methods used by multiple classes to backup HBase tables.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public final class BackupUtility {
-  protected static final Log LOG = LogFactory.getLog(BackupUtility.class);
-  public static final String LOGNAME_SEPARATOR = ".";
-
-  private BackupUtility(){
-    throw new AssertionError("Instantiating utility class...");
-  }
-
-  /**
-   * Check whether the backup path exist
-   * @param backupStr backup
-   * @param conf configuration
-   * @return Yes if path exists
-   * @throws IOException exception
-   */
-  public static boolean checkPathExist(String backupStr, Configuration conf)
-    throws IOException {
-    boolean isExist = false;
-    Path backupPath = new Path(backupStr);
-    FileSystem fileSys = backupPath.getFileSystem(conf);
-    String targetFsScheme = fileSys.getUri().getScheme();
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Schema of given url: " + backupStr + " is: " + targetFsScheme);
-    }
-    if (fileSys.exists(backupPath)) {
-      isExist = true;
-    }
-    return isExist;
-  }
-
-  // check target path first, confirm it doesn't exist before backup
-  public static void checkTargetDir(String backupRootPath, Configuration conf) throws IOException {
-    boolean targetExists = false;
-    try {
-      targetExists = checkPathExist(backupRootPath, conf);
-    } catch (IOException e) {
-      String expMsg = e.getMessage();
-      String newMsg = null;
-      if (expMsg.contains("No FileSystem for scheme")) {
-        newMsg =
-            "Unsupported filesystem scheme found in the backup target url. Error Message: "
-                + newMsg;
-        LOG.error(newMsg);
-        throw new IOException(newMsg);
-      } else {
-        throw e;
-      }
-    } catch (RuntimeException e) {
-      LOG.error(e.getMessage());
-      throw e;
-    }
-
-    if (targetExists) {
-      LOG.info("Using existing backup root dir: " + backupRootPath);
-    } else {
-      LOG.info("Backup root dir " + backupRootPath + " does not exist. Will be created.");
-    }
-  }
-
-  /**
-   * Get the min value for all the Values a map.
-   * @param map map
-   * @return the min value
-   */
-  public static <T> Long getMinValue(HashMap<T, Long> map) {
-    Long minTimestamp = null;
-    if (map != null) {
-      ArrayList<Long> timestampList = new ArrayList<Long>(map.values());
-      Collections.sort(timestampList);
-      // The min among all the RS log timestamps will be kept in hbase:backup table.
-      minTimestamp = timestampList.get(0);
-    }
-    return minTimestamp;
-  }
-
-  /**
-   * TODO: verify the code
-   * @param p path
-   * @return host name
-   * @throws IOException exception
-   */
-  public static String parseHostFromOldLog(Path p) throws IOException {
-    String n = p.getName();
-    int idx = n.lastIndexOf(LOGNAME_SEPARATOR);
-    String s = URLDecoder.decode(n.substring(0, idx), "UTF8");
-    return ServerName.parseHostname(s) + ":" + ServerName.parsePort(s);
-  }
-
-  /**
-   * Given the log file, parse the timestamp from the file name. The timestamp is the last number.
-   * @param p a path to the log file
-   * @return the timestamp
-   * @throws IOException exception
-   */
-  public static Long getCreationTime(Path p) throws IOException {
-    int idx = p.getName().lastIndexOf(LOGNAME_SEPARATOR);
-    if (idx < 0) {
-      throw new IOException("Cannot parse timestamp from path " + p);
-    }
-    String ts = p.getName().substring(idx + 1);
-    return Long.parseLong(ts);
-  }
-
-  public static List<String> getFiles(FileSystem fs, Path rootDir, List<String> files,
-    PathFilter filter) throws FileNotFoundException, IOException {
-    RemoteIterator<LocatedFileStatus> it = fs.listFiles(rootDir, true);
-
-    while (it.hasNext()) {
-      LocatedFileStatus lfs = it.next();
-      if (lfs.isDirectory()) {
-        continue;
-      }
-      // apply filter
-      if (filter.accept(lfs.getPath())) {
-        files.add(lfs.getPath().toString());
-      }
-    }
-    return files;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/524edfdb/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java
index 5b8a151..bf33cfb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupClient;
 import org.apache.hadoop.hbase.backup.BackupType;
-import org.apache.hadoop.hbase.backup.BackupUtility;
+import org.apache.hadoop.hbase.backup.BackupClientUtil;
 import org.apache.hadoop.hbase.backup.HBackupFileSystem;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
@@ -157,7 +157,7 @@ public final class BackupClientImpl implements BackupClient {
       throws IOException {
 
     String backupId = BackupRestoreConstants.BACKUPID_PREFIX + EnvironmentEdgeManager.currentTime();
-    BackupUtility.checkTargetDir(backupRootPath, conf);
+    BackupClientUtil.checkTargetDir(backupRootPath, conf);
 
     // table list specified for backup, trigger backup on specified tables
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/524edfdb/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java
index 7bd6e99..b9d71f6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupHandler.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
 import org.apache.hadoop.hbase.backup.BackupType;
-import org.apache.hadoop.hbase.backup.BackupUtility;
+import org.apache.hadoop.hbase.backup.BackupClientUtil;
 import org.apache.hadoop.hbase.backup.HBackupFileSystem;
 import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
 import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
@@ -195,7 +195,7 @@ public class BackupHandler implements Callable<Void> {
           backupManager.readLogTimestampMap();
 
       Long newStartCode =
-          BackupUtility.getMinValue(BackupUtil.getRSLogTimestampMins(newTableSetTimestampMap));
+          BackupClientUtil.getMinValue(BackupUtil.getRSLogTimestampMins(newTableSetTimestampMap));
       backupManager.writeBackupStartCode(newStartCode);
 
       // backup complete

http://git-wip-us.apache.org/repos/asf/hbase/blob/524edfdb/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupUtil.java
index 9981f78..96812c9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupUtil.java
@@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.HBackupFileSystem;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.backup.BackupUtility;
+import org.apache.hadoop.hbase.backup.BackupClientUtil;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -104,7 +104,7 @@ public final class BackupUtil {
     }
 
     for (String rs : rsLogTimestampMapByRS.keySet()) {
-      rsLogTimestamptMins.put(rs, BackupUtility.getMinValue(rsLogTimestampMapByRS.get(rs)));
+      rsLogTimestamptMins.put(rs, BackupClientUtil.getMinValue(rsLogTimestampMapByRS.get(rs)));
     }
 
     return rsLogTimestamptMins;
@@ -189,7 +189,7 @@ public final class BackupUtil {
    */
   public static String parseHostNameFromLogFile(Path p) throws IOException {
     if (isArchivedLogFile(p)) {
-      return BackupUtility.parseHostFromOldLog(p);
+      return BackupClientUtil.parseHostFromOldLog(p);
     } else {
       ServerName sname = DefaultWALProvider.getServerNameFromWALDirectoryName(p);
       return sname.getHostname() + ":" + sname.getPort();
@@ -368,8 +368,8 @@ public final class BackupUtil {
     List<String> logFiles = new ArrayList<String>();
 
     FileSystem fs = FileSystem.get(c);
-    logFiles = BackupUtility.getFiles(fs, logDir, logFiles, null);
-    logFiles = BackupUtility.getFiles(fs, oldLogDir, logFiles, null);
+    logFiles = BackupClientUtil.getFiles(fs, logDir, logFiles, null);
+    logFiles = BackupClientUtil.getFiles(fs, oldLogDir, logFiles, null);
     return logFiles;
   }
 
@@ -387,8 +387,8 @@ public final class BackupUtil {
     List<String> logFiles = new ArrayList<String>();
 
     FileSystem fs = FileSystem.get(c);
-    logFiles = BackupUtility.getFiles(fs, logDir, logFiles, filter);
-    logFiles = BackupUtility.getFiles(fs, oldLogDir, logFiles, filter);
+    logFiles = BackupClientUtil.getFiles(fs, logDir, logFiles, filter);
+    logFiles = BackupClientUtil.getFiles(fs, oldLogDir, logFiles, filter);
     return logFiles;
   }
 
@@ -415,7 +415,7 @@ public final class BackupUtil {
           }
           String host = parseHostNameFromLogFile(p);
           Long oldTimestamp = hostTimestampMap.get(host);
-          Long currentLogTS = BackupUtility.getCreationTime(p);
+          Long currentLogTS = BackupClientUtil.getCreationTime(p);
           return currentLogTS <= oldTimestamp;
         } catch (IOException e) {
           LOG.error(e);
@@ -424,8 +424,8 @@ public final class BackupUtil {
       }
     };
     FileSystem fs = FileSystem.get(c);
-    logFiles = BackupUtility.getFiles(fs, logDir, logFiles, filter);
-    logFiles = BackupUtility.getFiles(fs, oldLogDir, logFiles, filter);
+    logFiles = BackupClientUtil.getFiles(fs, logDir, logFiles, filter);
+    logFiles = BackupClientUtil.getFiles(fs, oldLogDir, logFiles, filter);
     return logFiles;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/524edfdb/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
index a3d124f..61e15c4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.BackupUtility;
+import org.apache.hadoop.hbase.backup.BackupClientUtil;
 import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
@@ -217,7 +217,7 @@ public class IncrementalBackupManager {
         }
         currentLogFile = log.getPath().toString();
         resultLogFiles.add(currentLogFile);
-        currentLogTS = BackupUtility.getCreationTime(log.getPath());
+        currentLogTS = BackupClientUtil.getCreationTime(log.getPath());
         // newestTimestamps is up-to-date with the current list of hosts
         // so newestTimestamps.get(host) will not be null.
         if (Long.valueOf(currentLogTS) > Long.valueOf(newestTimestamps.get(host))) {
@@ -237,8 +237,8 @@ public class IncrementalBackupManager {
         }
         continue;
       }
-      host = BackupUtility.parseHostFromOldLog(p);
-      currentLogTS = BackupUtility.getCreationTime(p);
+      host = BackupClientUtil.parseHostFromOldLog(p);
+      currentLogTS = BackupClientUtil.getCreationTime(p);
       oldTimeStamp = olderTimestamps.get(host);
       /*
        * It is possible that there is no old timestamp in hbase:backup for this host. At the time of
@@ -292,7 +292,7 @@ public class IncrementalBackupManager {
       }
       Long timestamp = null;
       try {
-        timestamp = BackupUtility.getCreationTime(path);
+        timestamp = BackupClientUtil.getCreationTime(path);
         return timestamp > Long.valueOf(lastBackupTS);
       } catch (IOException e) {
         LOG.warn("Cannot read timestamp of log file " + path);

http://git-wip-us.apache.org/repos/asf/hbase/blob/524edfdb/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java
index 2f8ab1b..6617565 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupType;
-import org.apache.hadoop.hbase.backup.BackupUtility;
+import org.apache.hadoop.hbase.backup.BackupClientUtil;
 import org.apache.hadoop.hbase.backup.HBackupFileSystem;
 import org.apache.hadoop.hbase.backup.RestoreClient;
 import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
@@ -155,7 +155,7 @@ public final class RestoreClientImpl implements RestoreClient {
       for (BackupImage image : imageSet) {
         String imageDir =
             HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(), table);
-        if (!BackupUtility.checkPathExist(imageDir, conf)) {
+        if (!BackupClientUtil.checkPathExist(imageDir, conf)) {
           LOG.error("ERROR: backup image does not exist: " + imageDir);
           isValid = false;
           break;