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:
parent
5fe43bfda4
commit
fa51414e90
|
@ -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...");
|
||||
|
|
|
@ -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)));
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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() +"'!");
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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()]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue