HBASE-7538 Improve snapshot related error and exception messages

git-svn-id: https://svn.apache.org/repos/asf/hbase/branches/hbase-7290@1445851 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jonathan Hsieh 2013-02-13 18:58:18 +00:00
parent e9d9eeaf7b
commit 6c8228e18b
8 changed files with 61 additions and 31 deletions

View File

@ -2445,7 +2445,8 @@ Server {
throw new ServiceException(e);
}
LOG.debug("Starting snapshot for:" + request);
LOG.debug("Submitting snapshot request for:" +
SnapshotDescriptionUtils.toString(request.getSnapshot()));
// get the snapshot information
SnapshotDescription snapshot = SnapshotDescriptionUtils.validate(request.getSnapshot(),
this.conf);
@ -2515,7 +2516,8 @@ Server {
@Override
public IsSnapshotDoneResponse isSnapshotDone(RpcController controller,
IsSnapshotDoneRequest request) throws ServiceException {
LOG.debug("Checking to see if snapshot from request:" + request + " is done");
LOG.debug("Checking to see if snapshot from request:" +
SnapshotDescriptionUtils.toString(request.getSnapshot()) + " is done");
try {
IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder();
boolean done = snapshotManager.isSnapshotDone(request.getSnapshot());

View File

@ -109,9 +109,9 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
// 2. let the CreateTableHandler add the regions to meta
return metaChanges.getRegionsToAdd();
} catch (Exception e) {
String msg = "clone snapshot=" + snapshot + " failed";
String msg = "clone snapshot=" + SnapshotDescriptionUtils.toString(snapshot) + " failed";
LOG.error(msg, e);
IOException rse = new RestoreSnapshotException(msg, e);
IOException rse = new RestoreSnapshotException(msg, e, snapshot);
// these handlers aren't futures so we need to register the error here.
this.monitor.receive(new ForeignException(NAME, rse));

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescriptio
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.snapshot.CopyRecoveredEditsTask;
import org.apache.hadoop.hbase.snapshot.ReferenceRegionHFilesTask;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.TableInfoCopyTask;
import org.apache.hadoop.hbase.snapshot.TakeSnapshotUtils;
import org.apache.hadoop.hbase.util.FSUtils;
@ -86,7 +87,7 @@ 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:"
+ snapshot);
+ SnapshotDescriptionUtils.toString(snapshot));
for (HRegionInfo regionInfo : regions) {
// 2.1 copy the regionInfo files to the snapshot
Path snapshotRegionDir = TakeSnapshotUtils.getRegionSnapshotDirectory(snapshot, rootDir,
@ -113,11 +114,13 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
monitor.rethrowException();
} catch (Exception e) {
// make sure we capture the exception to propagate back to the client later
String reason = "Failed due to exception:" + e.getMessage();
String reason = "Failed snapshot " + SnapshotDescriptionUtils.toString(snapshot)
+ " due to exception:" + e.getMessage();
ForeignException ee = new ForeignException(reason, e);
monitor.receive(ee);
} finally {
LOG.debug("Marking snapshot" + this.snapshot + " as finished.");
LOG.debug("Marking snapshot" + SnapshotDescriptionUtils.toString(snapshot)
+ " as finished.");
// 6. mark the timer as finished - even if we got an exception, we don't need to time the
// operation any further

View File

@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
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.master.MasterFileSystem;
@ -100,7 +99,7 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
this.masterServices.getTableDescriptors().add(hTableDescriptor);
// 2. Execute the on-disk Restore
LOG.debug("Starting restore snapshot=" + snapshot);
LOG.debug("Starting restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot));
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(
masterServices.getConfiguration(), fs,
@ -115,10 +114,11 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
MetaEditor.mutateRegions(catalogTracker, hrisToRemove, hris);
// At this point the restore is complete. Next step is enabling the table.
LOG.info("Restore snapshot=" + snapshot.getName() + " on table=" +
LOG.info("Restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot) + " on table=" +
Bytes.toString(tableName) + " completed!");
} catch (IOException e) {
String msg = "restore snapshot=" + snapshot + " failed. re-run the restore command.";
String msg = "restore snapshot=" + SnapshotDescriptionUtils.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);
@ -141,7 +141,8 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
public void cancel(String why) {
if (this.stopped) return;
this.stopped = true;
String msg = "Stopping restore snapshot=" + snapshot + " because: " + why;
String msg = "Stopping restore snapshot=" + SnapshotDescriptionUtils.toString(snapshot)
+ " because: " + why;
LOG.info(msg);
CancellationException ce = new CancellationException(why);
this.monitor.receive(new ForeignException(masterServices.getServerName().toString(), ce));

View File

@ -304,12 +304,14 @@ public class SnapshotManager implements Stoppable {
"No snapshot name passed in request, can't figure out which snapshot you want to check.");
}
String ssString = SnapshotDescriptionUtils.toString(expected);
// check to see if the sentinel exists
TakeSnapshotHandler handler = getTakeSnapshotHandler(expected);
if (handler == null) {
// doesn't exist, check if it is already completely done.
if (!isSnapshotCompleted(expected)) {
throw new UnknownSnapshotException("Snapshot:" + expected.getName()
throw new UnknownSnapshotException("Snapshot " + ssString
+ " is not currently running or one of the known completed snapshots.");
}
// was done, return true;
@ -320,15 +322,16 @@ public class SnapshotManager implements Stoppable {
try {
handler.rethrowException();
} catch (ForeignException e) {
throw new HBaseSnapshotException("Snapshot error from RS", e, expected);
throw new HBaseSnapshotException("Snapshot " + ssString + " had an error from RS", e,
expected);
}
// check to see if we are done
if (handler.isFinished()) {
LOG.debug("Snapshot '" + expected.getName() + "' has completed, notifying client.");
LOG.debug("Snapshot '" + ssString + "' has completed, notifying client.");
return true;
} else if (LOG.isDebugEnabled()) {
LOG.debug("Sentinel isn't finished with snapshot '" + expected.getName() + "'!");
LOG.debug("Snapshoting '" + ssString + "' is still in progress!");
}
return false;
}
@ -370,14 +373,18 @@ public class SnapshotManager implements Stoppable {
// make sure we aren't already running a snapshot
if (isTakingSnapshot()) {
throw new SnapshotCreationException("Already running another snapshot:"
+ this.handler.getSnapshot(), snapshot);
throw new SnapshotCreationException("Rejected taking "
+ SnapshotDescriptionUtils.toString(snapshot)
+ " because we are already running another snapshot "
+ SnapshotDescriptionUtils.toString(this.handler.getSnapshot()), snapshot);
}
// make sure we aren't running a restore on the same table
if (isRestoringTable(snapshot.getTable())) {
throw new SnapshotCreationException("Restore in progress on the same table snapshot:"
+ this.handler.getSnapshot(), snapshot);
throw new SnapshotCreationException("Rejected taking "
+ SnapshotDescriptionUtils.toString(snapshot)
+ " because we are already have a restore in progress on the same snapshot "
+ SnapshotDescriptionUtils.toString(this.handler.getSnapshot()), snapshot);
}
try {
@ -388,7 +395,7 @@ public class SnapshotManager implements Stoppable {
// recreate the working directory for the snapshot
if (!fs.mkdirs(workingDir)) {
throw new SnapshotCreationException("Couldn't create working directory (" + workingDir
+ ") for snapshot.", snapshot);
+ ") for snapshot" , snapshot);
}
} catch (HBaseSnapshotException e) {
throw e;
@ -481,7 +488,7 @@ public class SnapshotManager implements Stoppable {
if (assignmentMgr.getZKTable().isEnabledTable(snapshot.getTable())) {
LOG.debug("Table enabled, starting distributed snapshot.");
snapshotEnabledTable(snapshot);
LOG.debug("Started snapshot: " + snapshot);
LOG.debug("Started snapshot: " + SnapshotDescriptionUtils.toString(snapshot));
}
// For disabled table, snapshot is created by the master
else if (assignmentMgr.getZKTable().isDisabledTable(snapshot.getTable())) {

View File

@ -143,14 +143,14 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
// complete the snapshot, atomically moving from tmp to .snapshot dir.
completeSnapshot(this.snapshotDir, this.workingDir, this.fs);
} catch (Exception e) {
LOG.error("Got exception taking snapshot", e);
String reason = "Failed due to exception:" + e.getMessage();
LOG.error("Got exception taking snapshot", e);
String reason = "Failed taking snapshot " + SnapshotDescriptionUtils.toString(snapshot)
+ " due to exception:" + e.getMessage();
LOG.error(reason, e);
ForeignException ee = new ForeignException(reason, e);
monitor.receive(ee);
// need to mark this completed to close off and allow cleanup to happen.
cancel("Failed to take snapshot '" + snapshot.getName() + "' due to exception");
cancel("Failed to take snapshot '" + SnapshotDescriptionUtils.toString(snapshot)
+ "' due to exception");
} finally {
LOG.debug("Launching cleanup of working dir:" + workingDir);
try {
@ -188,15 +188,16 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
/**
* Snapshot the specified regions
*/
protected abstract void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regions) throws IOException,
KeeperException;
protected abstract void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regions)
throws IOException, KeeperException;
@Override
public void cancel(String why) {
if (finished) return;
this.finished = true;
LOG.info("Stop taking snapshot=" + snapshot + " because: " + why);
LOG.info("Stop taking snapshot=" + SnapshotDescriptionUtils.toString(snapshot) + " because: "
+ why);
CancellationException ce = new CancellationException(why);
monitor.receive(new ForeignException(master.getServerName().toString(), ce));
}

View File

@ -342,4 +342,19 @@ public class SnapshotDescriptionUtils {
+ ") to completed directory(" + finishedDir + ").", snapshot);
}
}
/**
* Returns a single line (no \n) representation of snapshot metadata. Use this instead of
* {@link SnapshotDescription#toString()}. We don't replace SnapshotDescrpition's toString
* because it is auto-generated by protoc.
* @param ssd
* @return Single line string with a summary of the snapshot parameters
*/
public static String toString(SnapshotDescription ssd) {
if (ssd == null) {
return null;
}
return "{ ss=" + ssd.getName() + " table=" + ssd.getTable()
+ " type=" + ssd.getType() + " }";
}
}

View File

@ -58,7 +58,8 @@ public class TableInfoCopyTask extends SnapshotTask {
public Void call() throws Exception {
LOG.debug("Running table info copy.");
this.rethrowException();
LOG.debug("Attempting to copy table info for snapshot:" + this.snapshot);
LOG.debug("Attempting to copy table info for snapshot:"
+ SnapshotDescriptionUtils.toString(this.snapshot));
// get the HTable descriptor
HTableDescriptor orig = FSTableDescriptors.getTableDescriptor(fs, rootDir,
Bytes.toBytes(this.snapshot.getTable()));