HBASE-22810 Initialize an separate ThreadPoolExecutor for taking/restoring snapshot (addendum - use the old config key) (#517)

This commit is contained in:
openinx 2019-08-22 11:33:51 +08:00 committed by huzheng
parent 538088f3c1
commit d26f0062d1
5 changed files with 55 additions and 11 deletions

View File

@ -1462,10 +1462,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.
}

View File

@ -1351,9 +1351,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

View File

@ -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) {

View File

@ -143,10 +143,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

View File

@ -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);
}
}