HBASE-7848 Use ZK-based read/write lock to make flush-type snapshot robust against table enable/disable/alter

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1460074 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Enis Soztutar 2013-03-23 01:47:17 +00:00
parent 28cfad9739
commit 9a528fbc17
4 changed files with 44 additions and 8 deletions

View File

@ -69,6 +69,11 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
timeoutInjector = TakeSnapshotUtils.getMasterTimerAndBindToMonitor(snapshot, conf, monitor);
}
@Override
public DisabledTableSnapshotHandler prepare() throws Exception {
return (DisabledTableSnapshotHandler) super.prepare();
}
// TODO consider parallelizing these operations since they are independent. Right now its just
// easier to keep them serial though
@Override

View File

@ -28,11 +28,11 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.exceptions.HBaseSnapshotException;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.procedure.Procedure;
import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.exceptions.HBaseSnapshotException;
import org.apache.hadoop.hbase.util.Pair;
import com.google.common.collect.Lists;
@ -54,6 +54,11 @@ public class EnabledTableSnapshotHandler extends TakeSnapshotHandler {
this.coordinator = manager.getCoordinator();
}
@Override
public EnabledTableSnapshotHandler prepare() throws Exception {
return (EnabledTableSnapshotHandler) super.prepare();
}
// TODO consider switching over to using regionnames, rather than server names. This would allow
// regions to migrate during a snapshot, and then be involved when they are ready. Still want to
// enforce a snapshot time constraints, but lets us be potentially a bit more robust.

View File

@ -428,10 +428,10 @@ public class SnapshotManager implements Stoppable {
throws HBaseSnapshotException {
TakeSnapshotHandler handler;
try {
handler = new EnabledTableSnapshotHandler(snapshot, master, this);
handler = new EnabledTableSnapshotHandler(snapshot, master, this).prepare();
this.executorService.submit(handler);
this.handler = handler;
} catch (IOException e) {
} catch (Exception e) {
// cleanup the working directory by trying to delete it from the fs.
Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
try {
@ -537,10 +537,10 @@ public class SnapshotManager implements Stoppable {
DisabledTableSnapshotHandler handler;
try {
handler = new DisabledTableSnapshotHandler(snapshot, this.master);
handler = new DisabledTableSnapshotHandler(snapshot, this.master).prepare();
this.executorService.submit(handler);
this.handler = handler;
} catch (IOException e) {
} catch (Exception e) {
// cleanup the working directory by trying to delete it from the fs.
Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
try {

View File

@ -37,13 +37,15 @@ import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
import org.apache.hadoop.hbase.exceptions.SnapshotCreationException;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.SnapshotSentinel;
import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.exceptions.SnapshotCreationException;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.TableInfoCopyTask;
import org.apache.hadoop.hbase.util.Bytes;
@ -54,7 +56,7 @@ import org.apache.zookeeper.KeeperException;
* A handler for taking snapshots from the master.
*
* This is not a subclass of TableEventHandler because using that would incur an extra META scan.
*
*
* The {@link #snapshotRegions(List)} call should get implemented for each snapshot flavor.
*/
@InterfaceAudience.Private
@ -74,6 +76,8 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
protected final Path workingDir;
private final MasterSnapshotVerifier verifier;
protected final ForeignExceptionDispatcher monitor;
protected final TableLockManager tableLockManager;
protected final TableLock tableLock;
/**
* @param snapshot descriptor of the snapshot to take
@ -95,7 +99,9 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
this.workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
this.monitor = new ForeignExceptionDispatcher();
loadTableDescriptor(); // check that .tableinfo is present
this.tableLockManager = master.getTableLockManager();
this.tableLock = this.tableLockManager.writeLock(Bytes.toBytes(snapshot.getTable())
, EventType.C_M_SNAPSHOT_TABLE.toString());
// prepare the verify
this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, rootDir);
@ -112,6 +118,15 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
return htd;
}
public TakeSnapshotHandler prepare() throws Exception {
super.prepare();
loadTableDescriptor(); // check that .tableinfo is present
this.tableLock.acquire(); //after this, you should ensure to release this lock in
//case of exceptions
return this;
}
/**
* Execute the core common portions of taking a snapshot. The {@link #snapshotRegions(List)}
* call should get implemented for each snapshot flavor.
@ -166,6 +181,17 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
} catch (IOException e) {
LOG.error("Couldn't delete snapshot working directory:" + workingDir);
}
releaseTableLock();
}
}
protected void releaseTableLock() {
if (this.tableLock != null) {
try {
this.tableLock.release();
} catch (IOException ex) {
LOG.warn("Could not release the table lock", ex);
}
}
}