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"); LOG.warn("Error on reopening the regions");
} }
} }
completed(null);
} catch (IOException e) { } catch (IOException e) {
LOG.error("Error manipulating table " + Bytes.toString(tableName), e); LOG.error("Error manipulating table " + Bytes.toString(tableName), e);
completed(e);
} catch (KeeperException e) { } catch (KeeperException e) {
LOG.error("Error manipulating table " + Bytes.toString(tableName), e); LOG.error("Error manipulating table " + Bytes.toString(tableName), e);
completed(e);
} finally { } finally {
releaseTableLock(); 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 { public boolean reOpenAllRegions(List<HRegionInfo> regions) throws IOException {
boolean done = false; boolean done = false;
LOG.info("Bucketing regions by region server..."); 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.MasterServices;
import org.apache.hadoop.hbase.master.SnapshotSentinel; import org.apache.hadoop.hbase.master.SnapshotSentinel;
import org.apache.hadoop.hbase.master.handler.CreateTableHandler; 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.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
@ -60,6 +62,7 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
private final SnapshotDescription snapshot; private final SnapshotDescription snapshot;
private final ForeignExceptionDispatcher monitor; private final ForeignExceptionDispatcher monitor;
private final MonitoredTask status;
private volatile boolean stopped = false; private volatile boolean stopped = false;
@ -74,6 +77,8 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
// Monitor // Monitor
this.monitor = new ForeignExceptionDispatcher(); this.monitor = new ForeignExceptionDispatcher();
this.status = TaskMonitor.get().createStatus("Cloning snapshot '" + snapshot.getName() +
"' to table " + hTableDescriptor.getNameAsString());
} }
@Override @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. * will be responsible to add the regions returned by this method to META and do the assignment.
*/ */
@Override @Override
protected List<HRegionInfo> handleCreateHdfsRegions(final Path tableRootDir, final String tableName) protected List<HRegionInfo> handleCreateHdfsRegions(final Path tableRootDir,
throws IOException { final String tableName) throws IOException {
status.setStatus("Creating regions for table: " + tableName);
FileSystem fs = fileSystemManager.getFileSystem(); FileSystem fs = fileSystemManager.getFileSystem();
Path rootDir = fileSystemManager.getRootDir(); Path rootDir = fileSystemManager.getRootDir();
Path tableDir = new Path(tableRootDir, tableName); Path tableDir = new Path(tableRootDir, tableName);
@ -98,7 +104,7 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
// 1. Execute the on-disk Clone // 1. Execute the on-disk Clone
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir); Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(conf, fs, RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(conf, fs,
snapshot, snapshotDir, hTableDescriptor, tableDir, monitor); snapshot, snapshotDir, hTableDescriptor, tableDir, monitor, status);
RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreHdfsRegions(); RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreHdfsRegions();
// Clone operation should not have stuff to restore or remove // 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"); "A clone should not have regions to remove");
// At this point the clone is complete. Next step is enabling the table. // 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 // 2. let the CreateTableHandler add the regions to meta
return metaChanges.getRegionsToAdd(); return metaChanges.getRegionsToAdd();
} catch (Exception e) { } 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); LOG.error(msg, e);
IOException rse = new RestoreSnapshotException(msg, e, snapshot); IOException rse = new RestoreSnapshotException(msg, e, snapshot);
@ -126,6 +135,12 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
@Override @Override
protected void completed(final Throwable exception) { protected void completed(final Throwable exception) {
this.stopped = true; 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); super.completed(exception);
} }
@ -143,7 +158,9 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
public void cancel(String why) { public void cancel(String why) {
if (this.stopped) return; if (this.stopped) return;
this.stopped = true; 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))); 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.ForeignException;
import org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector; import org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector;
import org.apache.hadoop.hbase.master.MasterServices; 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.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; 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 // TODO consider parallelizing these operations since they are independent. Right now its just
// easier to keep them serial though // easier to keep them serial though
@Override @Override
public void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regionsAndLocations) throws IOException, public void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regionsAndLocations)
KeeperException { throws IOException, KeeperException {
try { try {
timeoutInjector.start(); timeoutInjector.start();
@ -95,8 +97,10 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
} }
// 2. for each region, write all the info to disk // 2. for each region, write all the info to disk
LOG.info("Starting to write region info and WALs for regions for offline snapshot:" String msg = "Starting to write region info and WALs for regions for offline snapshot:"
+ ClientSnapshotDescriptionUtils.toString(snapshot)); + ClientSnapshotDescriptionUtils.toString(snapshot);
LOG.info(msg);
status.setStatus(msg);
for (HRegionInfo regionInfo : regions) { for (HRegionInfo regionInfo : regions) {
// 2.1 copy the regionInfo files to the snapshot // 2.1 copy the regionInfo files to the snapshot
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs,
@ -110,10 +114,14 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
Path snapshotRegionDir = regionFs.getRegionDir(); Path snapshotRegionDir = regionFs.getRegionDir();
new CopyRecoveredEditsTask(snapshot, monitor, fs, regionDir, snapshotRegionDir).call(); new CopyRecoveredEditsTask(snapshot, monitor, fs, regionDir, snapshotRegionDir).call();
monitor.rethrowException(); monitor.rethrowException();
status.setStatus("Completed copying recovered edits for offline snapshot of table: "
+ snapshot.getTable());
// 2.3 reference all the files in the region // 2.3 reference all the files in the region
new ReferenceRegionHFilesTask(snapshot, monitor, regionDir, fs, snapshotRegionDir).call(); new ReferenceRegionHFilesTask(snapshot, monitor, regionDir, fs, snapshotRegionDir).call();
monitor.rethrowException(); monitor.rethrowException();
status.setStatus("Completed referencing HFiles for offline snapshot of table: " +
snapshot.getTable());
} }
// 3. write the table info to disk // 3. write the table info to disk
@ -123,12 +131,14 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
FSUtils.getRootDir(conf)); FSUtils.getRootDir(conf));
tableInfoCopyTask.call(); tableInfoCopyTask.call();
monitor.rethrowException(); monitor.rethrowException();
status.setStatus("Finished copying tableinfo for snapshot of table: " + snapshot.getTable());
} catch (Exception e) { } catch (Exception e) {
// make sure we capture the exception to propagate back to the client later // make sure we capture the exception to propagate back to the client later
String reason = "Failed snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot) String reason = "Failed snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot)
+ " due to exception:" + e.getMessage(); + " due to exception:" + e.getMessage();
ForeignException ee = new ForeignException(reason, e); ForeignException ee = new ForeignException(reason, e);
monitor.receive(ee); monitor.receive(ee);
status.abort("Snapshot of table: "+ snapshot.getTable() +" failed because " + e.getMessage());
} finally { } finally {
LOG.debug("Marking snapshot" + ClientSnapshotDescriptionUtils.toString(snapshot) LOG.debug("Marking snapshot" + ClientSnapshotDescriptionUtils.toString(snapshot)
+ " as finished."); + " as finished.");
@ -138,4 +148,4 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
timeoutInjector.complete(); 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.MasterServices;
import org.apache.hadoop.hbase.master.SnapshotSentinel; import org.apache.hadoop.hbase.master.SnapshotSentinel;
import org.apache.hadoop.hbase.master.handler.TableEventHandler; 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.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.exceptions.RestoreSnapshotException; import org.apache.hadoop.hbase.exceptions.RestoreSnapshotException;
@ -60,6 +62,8 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
private final SnapshotDescription snapshot; private final SnapshotDescription snapshot;
private final ForeignExceptionDispatcher monitor; private final ForeignExceptionDispatcher monitor;
private final MonitoredTask status;
private volatile boolean stopped = false; private volatile boolean stopped = false;
public RestoreSnapshotHandler(final MasterServices masterServices, 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 is the new schema we are going to write out as this modification.
this.hTableDescriptor = htd; 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); Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper( RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(
masterServices.getConfiguration(), fs, masterServices.getConfiguration(), fs,
snapshot, snapshotDir, hTableDescriptor, tableDir, monitor); snapshot, snapshotDir, hTableDescriptor, tableDir, monitor, status);
RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreHdfsRegions(); RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreHdfsRegions();
// 3. Applies changes to .META. // 3. Applies changes to .META.
hris.clear(); hris.clear();
status.setStatus("Preparing to restore each region");
if (metaChanges.hasRegionsToAdd()) hris.addAll(metaChanges.getRegionsToAdd()); if (metaChanges.hasRegionsToAdd()) hris.addAll(metaChanges.getRegionsToAdd());
if (metaChanges.hasRegionsToRestore()) hris.addAll(metaChanges.getRegionsToRestore()); if (metaChanges.hasRegionsToRestore()) hris.addAll(metaChanges.getRegionsToRestore());
List<HRegionInfo> hrisToRemove = metaChanges.getRegionsToRemove(); List<HRegionInfo> hrisToRemove = metaChanges.getRegionsToRemove();
MetaEditor.mutateRegions(catalogTracker, hrisToRemove, hris); MetaEditor.mutateRegions(catalogTracker, hrisToRemove, hris);
// At this point the restore is complete. Next step is enabling the table. // At this point the restore is complete. Next step is enabling the table.
LOG.info("Restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) + " on table=" + LOG.info("Restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) +
Bytes.toString(tableName) + " completed!"); " on table=" + Bytes.toString(tableName) + " completed!");
} catch (IOException e) { } catch (IOException e) {
String msg = "restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) String msg = "restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot)
+ " failed. Try re-running the restore command."; + " failed. Try re-running the restore command.";
LOG.error(msg, e); LOG.error(msg, e);
monitor.receive(new ForeignException(masterServices.getServerName().toString(), e)); monitor.receive(new ForeignException(masterServices.getServerName().toString(), e));
throw new RestoreSnapshotException(msg, 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.executor.EventType;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.SnapshotSentinel; 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;
import org.apache.hadoop.hbase.master.TableLockManager.TableLock; import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; 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 ForeignExceptionDispatcher monitor;
protected final TableLockManager tableLockManager; protected final TableLockManager tableLockManager;
protected final TableLock tableLock; protected final TableLock tableLock;
protected final MonitoredTask status;
/** /**
* @param snapshot descriptor of the snapshot to take * @param snapshot descriptor of the snapshot to take
@ -105,6 +108,9 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
// prepare the verify // prepare the verify
this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, rootDir); 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() private HTableDescriptor loadTableDescriptor()
@ -133,7 +139,10 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
*/ */
@Override @Override
public void process() { 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 { try {
// If regions move after this meta scan, the region specific snapshot should fail, triggering // If regions move after this meta scan, the region specific snapshot should fail, triggering
// an external exception that gets captured here. // an external exception that gets captured here.
@ -157,11 +166,16 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
} }
// verify the snapshot is valid // verify the snapshot is valid
status.setStatus("Verifying snapshot: " + snapshot.getName());
verifier.verifySnapshot(this.workingDir, serverNames); verifier.verifySnapshot(this.workingDir, serverNames);
// complete the snapshot, atomically moving from tmp to .snapshot dir. // complete the snapshot, atomically moving from tmp to .snapshot dir.
completeSnapshot(this.snapshotDir, this.workingDir, this.fs); completeSnapshot(this.snapshotDir, this.workingDir, this.fs);
status.markComplete("Snapshot " + snapshot.getName() + " of table " + snapshot.getTable()
+ " completed");
} catch (Exception e) { } 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) String reason = "Failed taking snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot)
+ " due to exception:" + e.getMessage(); + " due to exception:" + e.getMessage();
LOG.error(reason, e); LOG.error(reason, e);
@ -226,8 +240,8 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
if (finished) return; if (finished) return;
this.finished = true; this.finished = true;
LOG.info("Stop taking snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) + " because: " LOG.info("Stop taking snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) +
+ why); " because: " + why);
CancellationException ce = new CancellationException(why); CancellationException ce = new CancellationException(why);
monitor.receive(new ForeignException(master.getServerName().toString(), ce)); 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.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.backup.HFileArchiver; 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.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
@ -102,6 +103,7 @@ public class RestoreSnapshotHelper {
new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR); new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
private final ForeignExceptionDispatcher monitor; private final ForeignExceptionDispatcher monitor;
private final MonitoredTask status;
private final SnapshotDescription snapshotDesc; private final SnapshotDescription snapshotDesc;
private final Path snapshotDir; private final Path snapshotDir;
@ -115,7 +117,7 @@ public class RestoreSnapshotHelper {
public RestoreSnapshotHelper(final Configuration conf, final FileSystem fs, public RestoreSnapshotHelper(final Configuration conf, final FileSystem fs,
final SnapshotDescription snapshotDescription, final Path snapshotDir, final SnapshotDescription snapshotDescription, final Path snapshotDir,
final HTableDescriptor tableDescriptor, final Path tableDir, final HTableDescriptor tableDescriptor, final Path tableDir,
final ForeignExceptionDispatcher monitor) final ForeignExceptionDispatcher monitor, final MonitoredTask status)
{ {
this.fs = fs; this.fs = fs;
this.conf = conf; this.conf = conf;
@ -124,6 +126,7 @@ public class RestoreSnapshotHelper {
this.tableDesc = tableDescriptor; this.tableDesc = tableDescriptor;
this.tableDir = tableDir; this.tableDir = tableDir;
this.monitor = monitor; this.monitor = monitor;
this.status = status;
} }
/** /**
@ -159,11 +162,15 @@ public class RestoreSnapshotHelper {
// Restore regions using the snapshot data // Restore regions using the snapshot data
monitor.rethrowException(); monitor.rethrowException();
status.setStatus("Restoring table regions...");
restoreHdfsRegions(metaChanges.getRegionsToRestore()); restoreHdfsRegions(metaChanges.getRegionsToRestore());
status.setStatus("Finished restoring all table regions.");
// Remove regions from the current table // Remove regions from the current table
monitor.rethrowException(); monitor.rethrowException();
status.setStatus("Starting to delete excess regions from table");
removeHdfsRegions(metaChanges.getRegionsToRemove()); removeHdfsRegions(metaChanges.getRegionsToRemove());
status.setStatus("Finished deleting excess regions from table.");
} }
// Regions to Add: present in the snapshot but not in the current 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 // Create new regions cloning from the snapshot
monitor.rethrowException(); monitor.rethrowException();
status.setStatus("Cloning regions...");
HRegionInfo[] clonedRegions = cloneHdfsRegions(regionsToAdd); HRegionInfo[] clonedRegions = cloneHdfsRegions(regionsToAdd);
metaChanges.setNewRegions(clonedRegions); metaChanges.setNewRegions(clonedRegions);
status.setStatus("Finished cloning regions.");
} }
// Restore WALs // Restore WALs
monitor.rethrowException(); monitor.rethrowException();
status.setStatus("Restoring WALs to table...");
restoreWALs(); restoreWALs();
status.setStatus("Finished restoring WALs to table.");
return metaChanges; 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.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.io.HFileLink; 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.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
@ -145,12 +146,13 @@ public class TestRestoreSnapshotHelper {
CatalogTracker catalogTracker = Mockito.mock(CatalogTracker.class); CatalogTracker catalogTracker = Mockito.mock(CatalogTracker.class);
HTableDescriptor tableDescriptor = Mockito.mock(HTableDescriptor.class); HTableDescriptor tableDescriptor = Mockito.mock(HTableDescriptor.class);
ForeignExceptionDispatcher monitor = Mockito.mock(ForeignExceptionDispatcher.class); ForeignExceptionDispatcher monitor = Mockito.mock(ForeignExceptionDispatcher.class);
MonitoredTask status = Mockito.mock(MonitoredTask.class);
SnapshotDescription sd = SnapshotDescription.newBuilder() SnapshotDescription sd = SnapshotDescription.newBuilder()
.setName("snapshot").setTable(sourceTableName).build(); .setName("snapshot").setTable(sourceTableName).build();
return new RestoreSnapshotHelper(conf, fs, sd, snapshotDir, 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) private void createSnapshot(final Path rootDir, final Path snapshotDir, final HTableDescriptor htd)
@ -199,4 +201,4 @@ public class TestRestoreSnapshotHelper {
Collections.sort(files); Collections.sort(files);
return files.toArray(new String[files.size()]); return files.toArray(new String[files.size()]);
} }
} }