HBASE-21480 Taking snapshot when RS crashes prevent we bring the regions online

This commit is contained in:
zhangduo 2018-11-17 14:40:15 +08:00
parent e5758e86a8
commit 297d49b680
7 changed files with 149 additions and 3 deletions

View File

@ -495,6 +495,13 @@ public class MergeTableRegionsProcedure
* @throws IOException
*/
private boolean prepareMergeRegion(final MasterProcedureEnv env) throws IOException {
// Fail if we are taking snapshot for the given table
if (env.getMasterServices().getSnapshotManager()
.isTakingSnapshot(regionsToMerge[0].getTable())) {
throw new MergeRegionException(
"Skip merging regions " + RegionInfo.getShortNameToLog(regionsToMerge) +
", because we are taking snapshot for the table " + regionsToMerge[0].getTable());
}
// Note: the following logic assumes that we only have 2 regions to merge. In the future,
// if we want to extend to more than 2 regions, the code needs to be modified a little bit.
CatalogJanitor catalogJanitor = env.getMasterServices().getCatalogJanitor();

View File

@ -465,6 +465,13 @@ public class SplitTableRegionProcedure
*/
@VisibleForTesting
public boolean prepareSplitRegion(final MasterProcedureEnv env) throws IOException {
// Fail if we are taking snapshot for the given table
if (env.getMasterServices().getSnapshotManager()
.isTakingSnapshot(getParentRegion().getTable())) {
setFailure(new IOException("Skip splitting region " + getParentRegion().getShortNameToLog() +
", because we are taking snapshot for the table " + getParentRegion().getTable()));
return false;
}
// Check whether the region is splittable
RegionStateNode node =
env.getAssignmentManager().getRegionStates().getRegionStateNode(getParentRegion());

View File

@ -119,4 +119,11 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
+ " as finished.");
}
}
@Override
protected boolean downgradeToSharedTableLock() {
// for taking snapshot on disabled table, it is OK to always hold the exclusive lock, as we do
// not need to assign the regions when there are region server crashes.
return false;
}
}

View File

@ -132,4 +132,12 @@ public class EnabledTableSnapshotHandler extends TakeSnapshotHandler {
monitor.rethrowException();
status.setStatus("Completed referencing HFiles for the mob region of table: " + snapshotTable);
}
@Override
protected boolean downgradeToSharedTableLock() {
// return true here to change from exclusive lock to shared lock, so we can still assign regions
// while taking snapshots. This is important, as region server crash can happen at any time, if
// we can not assign regions then the cluster will be in trouble as the regions can not online.
return true;
}
}

View File

@ -420,7 +420,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
* @param tableName name of the table being snapshotted.
* @return <tt>true</tt> if there is a snapshot in progress on the specified table.
*/
synchronized boolean isTakingSnapshot(final TableName tableName) {
public synchronized boolean isTakingSnapshot(final TableName tableName) {
SnapshotSentinel handler = this.snapshotHandlers.get(tableName);
return handler != null && !handler.isFinished();
}

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.MetricsSnapshot;
import org.apache.hadoop.hbase.master.SnapshotSentinel;
import org.apache.hadoop.hbase.master.locking.LockManager;
import org.apache.hadoop.hbase.master.locking.LockManager.MasterLock;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure2.LockType;
@ -83,7 +84,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
protected final Path workingDir;
private final MasterSnapshotVerifier verifier;
protected final ForeignExceptionDispatcher monitor;
protected final LockManager.MasterLock tableLock;
private final LockManager.MasterLock tableLock;
protected final MonitoredTask status;
protected final TableName snapshotTable;
protected final SnapshotManifest snapshotManifest;
@ -160,8 +161,16 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
+ eventType + " on table " + snapshotTable;
LOG.info(msg);
ReentrantLock lock = snapshotManager.getLocks().acquireLock(snapshot.getName());
MasterLock tableLockToRelease = this.tableLock;
status.setStatus(msg);
try {
if (downgradeToSharedTableLock()) {
// release the exclusive lock and hold the shared lock instead
tableLockToRelease = master.getLockManager().createMasterLock(snapshotTable,
LockType.SHARED, this.getClass().getName() + ": take snapshot " + snapshot.getName());
tableLock.release();
tableLockToRelease.acquire();
}
// If regions move after this meta scan, the region specific snapshot should fail, triggering
// an external exception that gets captured here.
@ -229,7 +238,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
LOG.error("Couldn't delete snapshot working directory:" + workingDir);
}
lock.unlock();
tableLock.release();
tableLockToRelease.release();
}
}
@ -253,6 +262,16 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
finished = true;
}
/**
* When taking snapshot, first we must acquire the exclusive table lock to confirm that there are
* no ongoing merge/split procedures. But later, we should try our best to release the exclusive
* lock as this may hurt the availability, because we need to hold the shared lock when assigning
* regions.
* <p/>
* See HBASE-21480 for more details.
*/
protected abstract boolean downgradeToSharedTableLock();
/**
* Snapshot the specified regions
*/

View File

@ -0,0 +1,98 @@
/**
* 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.master.snapshot;
import static org.junit.Assert.assertNull;
import java.io.IOException;
import java.io.UncheckedIOException;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.locking.LockManager.MasterLock;
import org.apache.hadoop.hbase.master.locking.LockProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.LockType;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({ MasterTests.class, MediumTests.class })
public class TestSnapshotWhileRSCrashes {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestSnapshotWhileRSCrashes.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static TableName NAME = TableName.valueOf("Cleanup");
private static byte[] CF = Bytes.toBytes("cf");
@BeforeClass
public static void setUp() throws Exception {
UTIL.startMiniCluster(3);
UTIL.createMultiRegionTable(NAME, CF);
UTIL.waitTableAvailable(NAME);
}
@AfterClass
public static void tearDown() throws Exception {
UTIL.shutdownMiniCluster();
}
@Test
public void test() throws InterruptedException, IOException {
String snName = "sn";
MasterLock lock = UTIL.getMiniHBaseCluster().getMaster().getLockManager().createMasterLock(NAME,
LockType.EXCLUSIVE, "for testing");
lock.acquire();
Thread t = new Thread(() -> {
try {
UTIL.getAdmin().snapshot(snName, NAME);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
});
t.setDaemon(true);
t.start();
ProcedureExecutor<MasterProcedureEnv> procExec =
UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
UTIL.waitFor(10000,
() -> procExec.getProcedures().stream().filter(p -> !p.isFinished())
.filter(p -> p instanceof LockProcedure).map(p -> (LockProcedure) p)
.filter(p -> NAME.equals(p.getTableName())).anyMatch(p -> !p.isLocked()));
UTIL.getMiniHBaseCluster().stopRegionServer(0);
lock.release();
// the snapshot can not work properly when there are rs crashes, so here we just want to make
// sure that the regions could online
try (Table table = UTIL.getConnection().getTable(NAME);
ResultScanner scanner = table.getScanner(CF)) {
assertNull(scanner.next());
}
}
}