HBASE-8341 RestoreSnapshotHandler.prepare() is not called by SnapshotManager and TakeSnapshotHandler should first acquire the table lock

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1468262 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Enis Soztutar 2013-04-15 22:49:43 +00:00
parent 3ba63ba0e7
commit 6d99781ddf
4 changed files with 31 additions and 10 deletions

View File

@ -31,13 +31,13 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.exceptions.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.exceptions.TableExistsException;
import org.apache.hadoop.hbase.exceptions.TableNotDisabledException;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.exceptions.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.exceptions.TableExistsException;
import org.apache.hadoop.hbase.exceptions.TableNotDisabledException;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.BulkReOpen;
@ -63,6 +63,7 @@ public abstract class TableEventHandler extends EventHandler {
protected final byte [] tableName;
protected final String tableNameStr;
protected TableLock tableLock;
private boolean isPrepareCalled = false;
public TableEventHandler(EventType eventType, byte [] tableName, Server server,
MasterServices masterServices) {
@ -97,6 +98,7 @@ public abstract class TableEventHandler extends EventHandler {
releaseTableLock();
}
}
this.isPrepareCalled = true;
return this;
}
@ -113,6 +115,11 @@ public abstract class TableEventHandler extends EventHandler {
@Override
public void process() {
if (!isPrepareCalled) {
//For proper table locking semantics, the implementor should ensure to call
//TableEventHandler.prepare() before calling process()
throw new RuntimeException("Implementation should have called prepare() first");
}
try {
LOG.info("Handling table operation " + eventType + " on table " +
Bytes.toString(tableName));

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.exceptions.RestoreSnapshotException;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices;
@ -44,7 +45,6 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.exceptions.RestoreSnapshotException;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.util.Bytes;
@ -91,6 +91,10 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
+ hTableDescriptor.getNameAsString());
}
public RestoreSnapshotHandler prepare() throws IOException {
return (RestoreSnapshotHandler) super.prepare();
}
/**
* The restore table is executed in place.
* - The on-disk data will be restored - reference files are put in place without moving data

View File

@ -706,6 +706,8 @@ public class SnapshotManager implements Stoppable {
final HTableDescriptor hTableDescriptor) throws HBaseSnapshotException {
String tableName = hTableDescriptor.getNameAsString();
// TODO: There is definite race condition for managing the single handler. We should fix
// and remove the limitation of single snapshot / restore at a time.
// make sure we aren't running a snapshot on the same table
if (isTakingSnapshot(tableName)) {
throw new RestoreSnapshotException("Snapshot in progress on the restore table=" + tableName);
@ -718,7 +720,7 @@ public class SnapshotManager implements Stoppable {
try {
RestoreSnapshotHandler handler =
new RestoreSnapshotHandler(master, snapshot, hTableDescriptor, metricsMaster);
new RestoreSnapshotHandler(master, snapshot, hTableDescriptor, metricsMaster).prepare();
this.executorService.submit(handler);
restoreHandlers.put(hTableDescriptor.getNameAsString(), handler);
} catch (Exception e) {

View File

@ -43,10 +43,10 @@ import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.MetricsMaster;
import org.apache.hadoop.hbase.master.SnapshotSentinel;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
@ -129,10 +129,18 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
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
boolean success = false;
try {
loadTableDescriptor(); // check that .tableinfo is present
success = true;
} finally {
if (!success) {
releaseTableLock();
}
}
this.tableLock.acquire(); //after this, you should ensure to release this lock in
//case of exceptions
return this;
}