HBASE-21480 Taking snapshot when RS crashes prevent we bring the regions online
This commit is contained in:
parent
e5758e86a8
commit
297d49b680
|
@ -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();
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue