You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2015/10/27 22:10:36 UTC

hbase git commit: HBASE-14680 Two configs for snapshot timeout and better defaults (Heng Chen)

Repository: hbase
Updated Branches:
  refs/heads/master c91bfff58 -> 16ff57bea


HBASE-14680 Two configs for snapshot timeout and better defaults (Heng Chen)


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

Branch: refs/heads/master
Commit: 16ff57bea94645aae30ba9b6bf4375b2eec202f1
Parents: c91bfff
Author: Enis Soztutar <en...@apache.org>
Authored: Tue Oct 27 11:35:46 2015 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Tue Oct 27 14:01:01 2015 -0700

----------------------------------------------------------------------
 .../src/main/resources/hbase-default.xml        | 14 ++++++++++
 .../snapshot/DisabledTableSnapshotHandler.java  | 29 --------------------
 .../hbase/master/snapshot/SnapshotManager.java  | 14 +++-------
 .../snapshot/RegionServerSnapshotManager.java   |  4 +--
 .../snapshot/SnapshotDescriptionUtils.java      | 24 ++++++++++++++--
 5 files changed, 41 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/16ff57be/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index 1654391..c64873f 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -1686,4 +1686,18 @@ possible configurations would overwhelm and obscure the important.
       The max number of threads used in MobCompactor.
     </description>
   </property>
+  <property>
+    <name>hbase.snapshot.master.timeout.millis</name>
+    <value>300000</value>
+    <description>
+       Timeout for master for the snapshot procedure execution
+    </description>
+   </property>
+     <property>
+    <name>hbase.snapshot.region.timeout</name>
+    <value>300000</value>
+    <description>
+       Timeout for regionservers to keep threads in snapshot request pool waiting
+    </description>
+   </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/hbase/blob/16ff57be/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
index 1f2ed0a..5d59229 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
@@ -26,20 +26,16 @@ import java.util.concurrent.ThreadPoolExecutor;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
-import org.apache.hadoop.hbase.errorhandling.ForeignExceptionListener;
-import org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -56,7 +52,6 @@ import org.apache.zookeeper.KeeperException;
 @InterfaceStability.Evolving
 public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
   private static final Log LOG = LogFactory.getLog(DisabledTableSnapshotHandler.class);
-  private final TimeoutExceptionInjector timeoutInjector;
 
   /**
    * @param snapshot descriptor of the snapshot to take
@@ -65,9 +60,6 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
   public DisabledTableSnapshotHandler(SnapshotDescription snapshot,
       final MasterServices masterServices) {
     super(snapshot, masterServices);
-
-    // setup the timer
-    timeoutInjector = getMasterTimerAndBindToMonitor(snapshot, conf, monitor);
   }
 
   @Override
@@ -81,8 +73,6 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
   public void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regionsAndLocations)
       throws IOException, KeeperException {
     try {
-      timeoutInjector.start();
-
       // 1. get all the regions hosting this table.
 
       // extract each pair to separate lists
@@ -127,25 +117,6 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
     } finally {
       LOG.debug("Marking snapshot" + ClientSnapshotDescriptionUtils.toString(snapshot)
           + " as finished.");
-
-      // 3. mark the timer as finished - even if we got an exception, we don't need to time the
-      // operation any further
-      timeoutInjector.complete();
     }
   }
-
-
-  /**
-   * Create a snapshot timer for the master which notifies the monitor when an error occurs
-   * @param snapshot snapshot to monitor
-   * @param conf configuration to use when getting the max snapshot life
-   * @param monitor monitor to notify when the snapshot life expires
-   * @return the timer to use update to signal the start and end of the snapshot
-   */
-  private TimeoutExceptionInjector getMasterTimerAndBindToMonitor(SnapshotDescription snapshot,
-      Configuration conf, ForeignExceptionListener monitor) {
-    long maxTime = SnapshotDescriptionUtils.getMaxMasterTimeout(conf, snapshot.getType(),
-      SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
-    return new TimeoutExceptionInjector(monitor, maxTime);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16ff57be/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
index 38aa2c0..d367d6e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
@@ -122,15 +122,6 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
    */
   private static final String SNAPSHOT_WAKE_MILLIS_KEY = "hbase.snapshot.master.wakeMillis";
 
-  /** By default, check to see if the snapshot is complete (ms) */
-  private static final int SNAPSHOT_TIMEOUT_MILLIS_DEFAULT = 60000;
-
-  /**
-   * Conf key for # of ms elapsed before injecting a snapshot timeout error when waiting for
-   * completion.
-   */
-  private static final String SNAPSHOT_TIMEOUT_MILLIS_KEY = "hbase.snapshot.master.timeoutMillis";
-
   /** Name of the operation to use in the controller */
   public static final String ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION = "online-snapshot";
 
@@ -1078,7 +1069,10 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
     // get the configuration for the coordinator
     Configuration conf = master.getConfiguration();
     long wakeFrequency = conf.getInt(SNAPSHOT_WAKE_MILLIS_KEY, SNAPSHOT_WAKE_MILLIS_DEFAULT);
-    long timeoutMillis = conf.getLong(SNAPSHOT_TIMEOUT_MILLIS_KEY, SNAPSHOT_TIMEOUT_MILLIS_DEFAULT);
+    long timeoutMillis = Math.max(conf.getLong(SnapshotDescriptionUtils.SNAPSHOT_TIMEOUT_MILLIS_KEY,
+                    SnapshotDescriptionUtils.SNAPSHOT_TIMEOUT_MILLIS_DEFAULT),
+            conf.getLong(SnapshotDescriptionUtils.MASTER_SNAPSHOT_TIMEOUT_MILLIS,
+                    SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME));
     int opThreads = conf.getInt(SNAPSHOT_POOL_THREADS_KEY, SNAPSHOT_POOL_THREADS_DEFAULT);
 
     // setup the default procedure coordinator

http://git-wip-us.apache.org/repos/asf/hbase/blob/16ff57be/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
index e08cf0e..537329a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
@@ -88,8 +88,8 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager {
 
   /** Conf key for max time to keep threads in snapshot request pool waiting */
   public static final String SNAPSHOT_TIMEOUT_MILLIS_KEY = "hbase.snapshot.region.timeout";
-  /** Keep threads alive in request pool for max of 60 seconds */
-  public static final long SNAPSHOT_TIMEOUT_MILLIS_DEFAULT = 60000;
+  /** Keep threads alive in request pool for max of 300 seconds */
+  public static final long SNAPSHOT_TIMEOUT_MILLIS_DEFAULT = 5 * 60000;
 
   /** Conf key for millis between checks to see if snapshot completed or if there are errors*/
   public static final String SNAPSHOT_REQUEST_WAKE_MILLIS_KEY = "hbase.snapshot.region.wakefrequency";

http://git-wip-us.apache.org/repos/asf/hbase/blob/16ff57be/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java
index 2fc5d83..79e7312 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java
@@ -105,10 +105,27 @@ public class SnapshotDescriptionUtils {
   /** Default value if no start time is specified */
   public static final long NO_SNAPSHOT_START_TIME_SPECIFIED = 0;
 
+
   public static final String MASTER_SNAPSHOT_TIMEOUT_MILLIS = "hbase.snapshot.master.timeout.millis";
 
-  /** By default, wait 60 seconds for a snapshot to complete */
-  public static final long DEFAULT_MAX_WAIT_TIME = 60000;
+  /** By default, wait 300 seconds for a snapshot to complete */
+  public static final long DEFAULT_MAX_WAIT_TIME = 60000 * 5 ;
+
+
+  /**
+   * By default, check to see if the snapshot is complete (ms)
+   * @deprecated Use {@link #DEFAULT_MAX_WAIT_TIME} instead.
+   * */
+  @Deprecated
+  public static final int SNAPSHOT_TIMEOUT_MILLIS_DEFAULT = 60000 * 5;
+
+  /**
+   * Conf key for # of ms elapsed before injecting a snapshot timeout error when waiting for
+   * completion.
+   * @deprecated Use {@link #MASTER_SNAPSHOT_TIMEOUT_MILLIS} instead.
+   */
+  @Deprecated
+  public static final String SNAPSHOT_TIMEOUT_MILLIS_KEY = "hbase.snapshot.master.timeoutMillis";
 
   private SnapshotDescriptionUtils() {
     // private constructor for utility class
@@ -128,7 +145,8 @@ public class SnapshotDescriptionUtils {
     default:
       confKey = MASTER_SNAPSHOT_TIMEOUT_MILLIS;
     }
-    return conf.getLong(confKey, defaultMaxWaitTime);
+    return Math.max(conf.getLong(confKey, defaultMaxWaitTime),
+        conf.getLong(SNAPSHOT_TIMEOUT_MILLIS_KEY, defaultMaxWaitTime));
   }
 
   /**