HBASE-7415 Add task information to snapshot operation (Jesse Yates)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1464427 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
mbertozzi 2013-04-04 10:50:17 +00:00
parent 5fe43bfda4
commit fa51414e90
7 changed files with 104 additions and 22 deletions

View File

@ -131,10 +131,13 @@ public abstract class TableEventHandler extends EventHandler {
LOG.warn("Error on reopening the regions");
}
}
completed(null);
} catch (IOException e) {
LOG.error("Error manipulating table " + Bytes.toString(tableName), e);
completed(e);
} catch (KeeperException e) {
LOG.error("Error manipulating table " + Bytes.toString(tableName), e);
completed(e);
} finally {
releaseTableLock();
}
@ -150,6 +153,13 @@ public abstract class TableEventHandler extends EventHandler {
}
}
/**
* Called after that process() is completed.
* @param exception null if process() is successful or not null if something has failed.
*/
protected void completed(final Throwable exception) {
}
public boolean reOpenAllRegions(List<HRegionInfo> regions) throws IOException {
boolean done = false;
LOG.info("Bucketing regions by region server...");

View File

@ -38,6 +38,8 @@ import org.apache.hadoop.hbase.exceptions.TableExistsException;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.SnapshotSentinel;
import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
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.RestoreSnapshotHelper;
@ -60,6 +62,7 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
private final SnapshotDescription snapshot;
private final ForeignExceptionDispatcher monitor;
private final MonitoredTask status;
private volatile boolean stopped = false;
@ -74,6 +77,8 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
// Monitor
this.monitor = new ForeignExceptionDispatcher();
this.status = TaskMonitor.get().createStatus("Cloning snapshot '" + snapshot.getName() +
"' to table " + hTableDescriptor.getNameAsString());
}
@Override
@ -88,8 +93,9 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
* will be responsible to add the regions returned by this method to META and do the assignment.
*/
@Override
protected List<HRegionInfo> handleCreateHdfsRegions(final Path tableRootDir, final String tableName)
throws IOException {
protected List<HRegionInfo> handleCreateHdfsRegions(final Path tableRootDir,
final String tableName) throws IOException {
status.setStatus("Creating regions for table: " + tableName);
FileSystem fs = fileSystemManager.getFileSystem();
Path rootDir = fileSystemManager.getRootDir();
Path tableDir = new Path(tableRootDir, tableName);
@ -98,7 +104,7 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
// 1. Execute the on-disk Clone
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(conf, fs,
snapshot, snapshotDir, hTableDescriptor, tableDir, monitor);
snapshot, snapshotDir, hTableDescriptor, tableDir, monitor, status);
RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreHdfsRegions();
// Clone operation should not have stuff to restore or remove
@ -108,12 +114,15 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
"A clone should not have regions to remove");
// At this point the clone is complete. Next step is enabling the table.
LOG.info("Clone snapshot=" + snapshot.getName() + " on table=" + tableName + " completed!");
String msg = "Clone snapshot="+ snapshot.getName() +" on table=" + tableName + " completed!";
LOG.info(msg);
status.setStatus(msg + " Waiting for table to be enabled...");
// 2. let the CreateTableHandler add the regions to meta
return metaChanges.getRegionsToAdd();
} catch (Exception e) {
String msg = "clone snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) + " failed";
String msg = "clone snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) +
" failed because " + e.getMessage();
LOG.error(msg, e);
IOException rse = new RestoreSnapshotException(msg, e, snapshot);
@ -126,6 +135,12 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
@Override
protected void completed(final Throwable exception) {
this.stopped = true;
if (exception != null) {
status.abort("Snapshot '" + snapshot.getName() + "' clone failed because " +
exception.getMessage());
} else {
status.markComplete("Snapshot '"+ snapshot.getName() +"' clone completed and table enabled!");
}
super.completed(exception);
}
@ -143,7 +158,9 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
public void cancel(String why) {
if (this.stopped) return;
this.stopped = true;
LOG.info("Stopping clone snapshot=" + snapshot + " because: " + why);
String msg = "Stopping clone snapshot=" + snapshot + " because: " + why;
LOG.info(msg);
status.abort(msg);
this.monitor.receive(new ForeignException(NAME, new CancellationException(why)));
}

View File

@ -32,6 +32,8 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector;
import org.apache.hadoop.hbase.master.MasterServices;
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.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
@ -77,8 +79,8 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
// TODO consider parallelizing these operations since they are independent. Right now its just
// easier to keep them serial though
@Override
public void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regionsAndLocations) throws IOException,
KeeperException {
public void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regionsAndLocations)
throws IOException, KeeperException {
try {
timeoutInjector.start();
@ -95,8 +97,10 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
}
// 2. for each region, write all the info to disk
LOG.info("Starting to write region info and WALs for regions for offline snapshot:"
+ ClientSnapshotDescriptionUtils.toString(snapshot));
String msg = "Starting to write region info and WALs for regions for offline snapshot:"
+ ClientSnapshotDescriptionUtils.toString(snapshot);
LOG.info(msg);
status.setStatus(msg);
for (HRegionInfo regionInfo : regions) {
// 2.1 copy the regionInfo files to the snapshot
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs,
@ -110,10 +114,14 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
Path snapshotRegionDir = regionFs.getRegionDir();
new CopyRecoveredEditsTask(snapshot, monitor, fs, regionDir, snapshotRegionDir).call();
monitor.rethrowException();
status.setStatus("Completed copying recovered edits for offline snapshot of table: "
+ snapshot.getTable());
// 2.3 reference all the files in the region
new ReferenceRegionHFilesTask(snapshot, monitor, regionDir, fs, snapshotRegionDir).call();
monitor.rethrowException();
status.setStatus("Completed referencing HFiles for offline snapshot of table: " +
snapshot.getTable());
}
// 3. write the table info to disk
@ -123,12 +131,14 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
FSUtils.getRootDir(conf));
tableInfoCopyTask.call();
monitor.rethrowException();
status.setStatus("Finished copying tableinfo for snapshot of table: " + snapshot.getTable());
} catch (Exception e) {
// make sure we capture the exception to propagate back to the client later
String reason = "Failed snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot)
+ " due to exception:" + e.getMessage();
ForeignException ee = new ForeignException(reason, e);
monitor.receive(ee);
status.abort("Snapshot of table: "+ snapshot.getTable() +" failed because " + e.getMessage());
} finally {
LOG.debug("Marking snapshot" + ClientSnapshotDescriptionUtils.toString(snapshot)
+ " as finished.");
@ -138,4 +148,4 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
timeoutInjector.complete();
}
}
}
}

View File

@ -39,6 +39,8 @@ import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.SnapshotSentinel;
import org.apache.hadoop.hbase.master.handler.TableEventHandler;
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;
@ -60,6 +62,8 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
private final SnapshotDescription snapshot;
private final ForeignExceptionDispatcher monitor;
private final MonitoredTask status;
private volatile boolean stopped = false;
public RestoreSnapshotHandler(final MasterServices masterServices,
@ -78,6 +82,10 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
// This is the new schema we are going to write out as this modification.
this.hTableDescriptor = htd;
this.status = TaskMonitor.get().createStatus(
"Restoring snapshot '" + snapshot.getName() + "' to table "
+ hTableDescriptor.getNameAsString());
}
/**
@ -106,27 +114,37 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(
masterServices.getConfiguration(), fs,
snapshot, snapshotDir, hTableDescriptor, tableDir, monitor);
snapshot, snapshotDir, hTableDescriptor, tableDir, monitor, status);
RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreHdfsRegions();
// 3. Applies changes to .META.
hris.clear();
status.setStatus("Preparing to restore each region");
if (metaChanges.hasRegionsToAdd()) hris.addAll(metaChanges.getRegionsToAdd());
if (metaChanges.hasRegionsToRestore()) hris.addAll(metaChanges.getRegionsToRestore());
List<HRegionInfo> hrisToRemove = metaChanges.getRegionsToRemove();
MetaEditor.mutateRegions(catalogTracker, hrisToRemove, hris);
// At this point the restore is complete. Next step is enabling the table.
LOG.info("Restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) + " on table=" +
Bytes.toString(tableName) + " completed!");
LOG.info("Restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) +
" on table=" + Bytes.toString(tableName) + " completed!");
} catch (IOException e) {
String msg = "restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot)
+ " failed. Try re-running the restore command.";
LOG.error(msg, e);
monitor.receive(new ForeignException(masterServices.getServerName().toString(), e));
throw new RestoreSnapshotException(msg, e);
} finally {
this.stopped = true;
}
}
@Override
protected void completed(final Throwable exception) {
this.stopped = true;
if (exception != null) {
status.abort("Restore snapshot '" + snapshot.getName() + "' failed because " +
exception.getMessage());
} else {
status.markComplete("Restore snapshot '"+ snapshot.getName() +"'!");
}
}

View File

@ -42,6 +42,8 @@ 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.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.protobuf.generated.HBaseProtos.SnapshotDescription;
@ -78,6 +80,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
protected final ForeignExceptionDispatcher monitor;
protected final TableLockManager tableLockManager;
protected final TableLock tableLock;
protected final MonitoredTask status;
/**
* @param snapshot descriptor of the snapshot to take
@ -105,6 +108,9 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
// prepare the verify
this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, rootDir);
// update the running tasks
this.status = TaskMonitor.get().createStatus(
"Taking " + snapshot.getType() + " snapshot on table: " + snapshot.getTable());
}
private HTableDescriptor loadTableDescriptor()
@ -133,7 +139,10 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
*/
@Override
public void process() {
LOG.info("Running table snapshot operation " + eventType + " on table " + snapshot.getTable());
String msg = "Running " + snapshot.getType() + " table snapshot " + snapshot.getName() + " "
+ eventType + " on table " + snapshot.getTable();
LOG.info(msg);
status.setStatus(msg);
try {
// If regions move after this meta scan, the region specific snapshot should fail, triggering
// an external exception that gets captured here.
@ -157,11 +166,16 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
}
// verify the snapshot is valid
status.setStatus("Verifying snapshot: " + snapshot.getName());
verifier.verifySnapshot(this.workingDir, serverNames);
// complete the snapshot, atomically moving from tmp to .snapshot dir.
completeSnapshot(this.snapshotDir, this.workingDir, this.fs);
status.markComplete("Snapshot " + snapshot.getName() + " of table " + snapshot.getTable()
+ " completed");
} catch (Exception e) {
status.abort("Failed to complete snapshot " + snapshot.getName() + " on table " +
snapshot.getTable() + " because " + e.getMessage());
String reason = "Failed taking snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot)
+ " due to exception:" + e.getMessage();
LOG.error(reason, e);
@ -226,8 +240,8 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
if (finished) return;
this.finished = true;
LOG.info("Stop taking snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) + " because: "
+ why);
LOG.info("Stop taking snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) +
" because: " + why);
CancellationException ce = new CancellationException(why);
monitor.receive(new ForeignException(master.getServerName().toString(), ce));
}

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
@ -102,6 +103,7 @@ public class RestoreSnapshotHelper {
new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
private final ForeignExceptionDispatcher monitor;
private final MonitoredTask status;
private final SnapshotDescription snapshotDesc;
private final Path snapshotDir;
@ -115,7 +117,7 @@ public class RestoreSnapshotHelper {
public RestoreSnapshotHelper(final Configuration conf, final FileSystem fs,
final SnapshotDescription snapshotDescription, final Path snapshotDir,
final HTableDescriptor tableDescriptor, final Path tableDir,
final ForeignExceptionDispatcher monitor)
final ForeignExceptionDispatcher monitor, final MonitoredTask status)
{
this.fs = fs;
this.conf = conf;
@ -124,6 +126,7 @@ public class RestoreSnapshotHelper {
this.tableDesc = tableDescriptor;
this.tableDir = tableDir;
this.monitor = monitor;
this.status = status;
}
/**
@ -159,11 +162,15 @@ public class RestoreSnapshotHelper {
// Restore regions using the snapshot data
monitor.rethrowException();
status.setStatus("Restoring table regions...");
restoreHdfsRegions(metaChanges.getRegionsToRestore());
status.setStatus("Finished restoring all table regions.");
// Remove regions from the current table
monitor.rethrowException();
status.setStatus("Starting to delete excess regions from table");
removeHdfsRegions(metaChanges.getRegionsToRemove());
status.setStatus("Finished deleting excess regions from table.");
}
// Regions to Add: present in the snapshot but not in the current table
@ -179,13 +186,17 @@ public class RestoreSnapshotHelper {
// Create new regions cloning from the snapshot
monitor.rethrowException();
status.setStatus("Cloning regions...");
HRegionInfo[] clonedRegions = cloneHdfsRegions(regionsToAdd);
metaChanges.setNewRegions(clonedRegions);
status.setStatus("Finished cloning regions.");
}
// Restore WALs
monitor.rethrowException();
status.setStatus("Restoring WALs to table...");
restoreWALs();
status.setStatus("Finished restoring WALs to table.");
return metaChanges;
}

View File

@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
@ -145,12 +146,13 @@ public class TestRestoreSnapshotHelper {
CatalogTracker catalogTracker = Mockito.mock(CatalogTracker.class);
HTableDescriptor tableDescriptor = Mockito.mock(HTableDescriptor.class);
ForeignExceptionDispatcher monitor = Mockito.mock(ForeignExceptionDispatcher.class);
MonitoredTask status = Mockito.mock(MonitoredTask.class);
SnapshotDescription sd = SnapshotDescription.newBuilder()
.setName("snapshot").setTable(sourceTableName).build();
return new RestoreSnapshotHelper(conf, fs, sd, snapshotDir,
htdClone, HTableDescriptor.getTableDir(rootDir, htdClone.getName()), monitor);
htdClone, HTableDescriptor.getTableDir(rootDir, htdClone.getName()), monitor, status);
}
private void createSnapshot(final Path rootDir, final Path snapshotDir, final HTableDescriptor htd)
@ -199,4 +201,4 @@ public class TestRestoreSnapshotHelper {
Collections.sort(files);
return files.toArray(new String[files.size()]);
}
}
}