You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by op...@apache.org on 2019/08/22 03:54:58 UTC

[hbase] branch branch-2 updated: HBASE-22810 Initialize an separate ThreadPoolExecutor for taking/restoring snapshot (addendum - use the old config key) (#517)

This is an automated email from the ASF dual-hosted git repository.

openinx pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 53522f3  HBASE-22810 Initialize an separate ThreadPoolExecutor for taking/restoring snapshot (addendum - use the old config key) (#517)
53522f3 is described below

commit 53522f36cc3e40b424468d5df0846e68665dce7c
Author: openinx <op...@gmail.com>
AuthorDate: Thu Aug 22 11:33:51 2019 +0800

    HBASE-22810 Initialize an separate ThreadPoolExecutor for taking/restoring snapshot (addendum - use the old config key) (#517)
---
 .../java/org/apache/hadoop/hbase/HConstants.java   |  5 ---
 .../org/apache/hadoop/hbase/master/HMaster.java    |  5 +--
 .../snapshot/EnabledTableSnapshotHandler.java      |  3 +-
 .../hbase/master/snapshot/SnapshotManager.java     |  4 +-
 .../TestConcurrentFlushSnapshotFromClient.java     | 50 ++++++++++++++++++++++
 5 files changed, 55 insertions(+), 12 deletions(-)

diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index b78ce0f..f7fcccd 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -1526,11 +1526,6 @@ public final class HConstants {
       "hbase.master.executor.logreplayops.threads";
   public static final int MASTER_LOG_REPLAY_OPS_THREADS_DEFAULT = 10;
 
-  public static final String MASTER_SNAPSHOT_OPERATIONS_THREADS =
-      "hbase.master.executor.snapshot.threads";
-  public static final int MASTER_SNAPSHOT_OPERATIONS_THREADS_DEFAULT = 3;
-
-
   private HConstants() {
     // Can't be instantiated with this ctor.
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 0d3e5d6..4e445ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -1412,9 +1412,8 @@ public class HMaster extends HRegionServer implements MasterServices {
         HConstants.MASTER_META_SERVER_OPERATIONS_THREADS_DEFAULT));
     this.executorService.startExecutorService(ExecutorType.M_LOG_REPLAY_OPS, conf.getInt(
       HConstants.MASTER_LOG_REPLAY_OPS_THREADS, HConstants.MASTER_LOG_REPLAY_OPS_THREADS_DEFAULT));
-    this.executorService.startExecutorService(ExecutorType.MASTER_SNAPSHOT_OPERATIONS,
-      conf.getInt(HConstants.MASTER_SNAPSHOT_OPERATIONS_THREADS,
-        HConstants.MASTER_SNAPSHOT_OPERATIONS_THREADS_DEFAULT));
+    this.executorService.startExecutorService(ExecutorType.MASTER_SNAPSHOT_OPERATIONS, conf.getInt(
+      SnapshotManager.SNAPSHOT_POOL_THREADS_KEY, SnapshotManager.SNAPSHOT_POOL_THREADS_DEFAULT));
 
    // We depend on there being only one instance of this executor running
    // at a time.  To do concurrency, would need fencing of enable/disable of
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/EnabledTableSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/EnabledTableSnapshotHandler.java
index 3a6a813..81aa2df 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/EnabledTableSnapshotHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/EnabledTableSnapshotHandler.java
@@ -69,8 +69,7 @@ public class EnabledTableSnapshotHandler extends TakeSnapshotHandler {
    * phases to complete.
    */
   @Override
-  protected void snapshotRegions(List<Pair<RegionInfo, ServerName>> regions)
-      throws HBaseSnapshotException, IOException {
+  protected void snapshotRegions(List<Pair<RegionInfo, ServerName>> regions) throws IOException {
     Set<String> regionServers = new HashSet<>(regions.size());
     for (Pair<RegionInfo, ServerName> region : regions) {
       if (region != null && region.getFirst() != null && region.getSecond() != null) {
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 c813762..1b4f9d8 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
@@ -146,10 +146,10 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
   public static final String ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION = "online-snapshot";
 
   /** Conf key for # of threads used by the SnapshotManager thread pool */
-  private static final String SNAPSHOT_POOL_THREADS_KEY = "hbase.snapshot.master.threads";
+  public static final String SNAPSHOT_POOL_THREADS_KEY = "hbase.snapshot.master.threads";
 
   /** number of current operations running on the master */
-  private static final int SNAPSHOT_POOL_THREADS_DEFAULT = 1;
+  public static final int SNAPSHOT_POOL_THREADS_DEFAULT = 1;
 
   private boolean stopped;
   private MasterServices master;  // Needed by TableEventHandlers
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestConcurrentFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestConcurrentFlushSnapshotFromClient.java
new file mode 100644
index 0000000..9887998
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestConcurrentFlushSnapshotFromClient.java
@@ -0,0 +1,50 @@
+/**
+ * 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.snapshot;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ ClientTests.class, LargeTests.class })
+public class TestConcurrentFlushSnapshotFromClient extends TestFlushSnapshotFromClient {
+  private static final Logger LOG = LoggerFactory.getLogger(TestFlushSnapshotFromClient.class);
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestConcurrentFlushSnapshotFromClient.class);
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    setupConf(UTIL.getConfiguration());
+    UTIL.startMiniCluster(3);
+  }
+
+  protected static void setupConf(Configuration conf) {
+    TestFlushSnapshotFromClient.setupConf(conf);
+    UTIL.getConfiguration().setInt(SnapshotManager.SNAPSHOT_POOL_THREADS_KEY, 3);
+    LOG.info("Config the {} to be 3", SnapshotManager.SNAPSHOT_POOL_THREADS_KEY);
+  }
+}