HBASE-7615 Add metrics for snapshots

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1464441 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
mbertozzi 2013-04-04 11:10:56 +00:00
parent fa51414e90
commit 8465f20316
14 changed files with 390 additions and 71 deletions

View File

@ -57,6 +57,9 @@ public interface MetricsMasterSource extends BaseSource {
static final String IS_ACTIVE_MASTER_NAME = "isActiveMaster";
static final String SPLIT_TIME_NAME = "hlogSplitTime";
static final String SPLIT_SIZE_NAME = "hlogSplitSize";
static final String SNAPSHOT_TIME_NAME = "snapshotTime";
static final String SNAPSHOT_RESTORE_TIME_NAME = "snapshotRestoreTime";
static final String SNAPSHOT_CLONE_TIME_NAME = "snapshotCloneTime";
static final String CLUSTER_REQUESTS_NAME = "clusterRequests";
static final String RIT_COUNT_NAME = "ritCount";
static final String RIT_COUNT_OVER_THRESHOLD_NAME = "ritCountOverThreshold";
@ -72,6 +75,9 @@ public interface MetricsMasterSource extends BaseSource {
static final String IS_ACTIVE_MASTER_DESC = "Is Active Master";
static final String SPLIT_TIME_DESC = "Time it takes to finish HLog.splitLog()";
static final String SPLIT_SIZE_DESC = "Size of HLog files being split";
static final String SNAPSHOT_TIME_DESC = "Time it takes to finish snapshot()";
static final String SNAPSHOT_RESTORE_TIME_DESC = "Time it takes to finish restoreSnapshot()";
static final String SNAPSHOT_CLONE_TIME_DESC = "Time it takes to finish cloneSnapshot()";
/**
@ -106,4 +112,9 @@ public interface MetricsMasterSource extends BaseSource {
void updateSplitSize(long size);
void updateSnapshotTime(long time);
void updateSnapshotCloneTime(long time);
void updateSnapshotRestoreTime(long time);
}

View File

@ -26,6 +26,7 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.lib.MetricMutableCounterLong;
import org.apache.hadoop.metrics2.lib.MetricMutableGaugeLong;
import org.apache.hadoop.metrics2.lib.MetricMutableHistogram;
import org.apache.hadoop.metrics2.lib.MetricMutableStat;
/**
* Hadoop1 implementation of MetricsMasterSource.
@ -44,6 +45,9 @@ public class MetricsMasterSourceImpl
private MetricMutableGaugeLong ritOldestAgeGauge;
private MetricMutableHistogram splitTimeHisto;
private MetricMutableHistogram splitSizeHisto;
private MetricMutableStat snapshotTimeHisto;
private MetricMutableStat snapshotCloneTimeHisto;
private MetricMutableStat snapshotRestoreTimeHisto;
public MetricsMasterSourceImpl(MetricsMasterWrapper masterWrapper) {
this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT, masterWrapper);
@ -67,6 +71,12 @@ public class MetricsMasterSourceImpl
ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, "", 0l);
splitSizeHisto = metricsRegistry.newHistogram(SPLIT_SIZE_NAME, SPLIT_SIZE_DESC);
splitTimeHisto = metricsRegistry.newHistogram(SPLIT_TIME_NAME, SPLIT_TIME_DESC);
snapshotTimeHisto = metricsRegistry.newStat(
SNAPSHOT_TIME_NAME, SNAPSHOT_TIME_DESC, "Ops", "Time", true);
snapshotCloneTimeHisto = metricsRegistry.newStat(
SNAPSHOT_CLONE_TIME_NAME, SNAPSHOT_CLONE_TIME_DESC, "Ops", "Time", true);
snapshotRestoreTimeHisto = metricsRegistry.newStat(
SNAPSHOT_RESTORE_TIME_NAME, SNAPSHOT_RESTORE_TIME_DESC, "Ops", "Time", true);
}
public void incRequests(final int inc) {
@ -95,6 +105,21 @@ public class MetricsMasterSourceImpl
splitSizeHisto.add(size);
}
@Override
public void updateSnapshotTime(long time) {
snapshotTimeHisto.add(time);
}
@Override
public void updateSnapshotCloneTime(long time) {
snapshotCloneTimeHisto.add(time);
}
@Override
public void updateSnapshotRestoreTime(long time) {
snapshotRestoreTimeHisto.add(time);
}
/**
* Method to export all the metrics.
*

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.metrics2.lib.Interns;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
import org.apache.hadoop.metrics2.lib.MutableHistogram;
import org.apache.hadoop.metrics2.lib.MutableStat;
/**
* Hadoop2 implementation of MetricsMasterSource.
@ -41,6 +42,9 @@ public class MetricsMasterSourceImpl
private MutableGaugeLong ritOldestAgeGauge;
private MutableHistogram splitTimeHisto;
private MutableHistogram splitSizeHisto;
private MutableStat snapshotTimeHisto;
private MutableStat snapshotCloneTimeHisto;
private MutableStat snapshotRestoreTimeHisto;
public MetricsMasterSourceImpl(MetricsMasterWrapper masterWrapper) {
this(METRICS_NAME,
@ -69,6 +73,12 @@ public class MetricsMasterSourceImpl
ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, "", 0l);
splitSizeHisto = metricsRegistry.newHistogram(SPLIT_SIZE_NAME, SPLIT_SIZE_DESC);
splitTimeHisto = metricsRegistry.newHistogram(SPLIT_TIME_NAME, SPLIT_TIME_DESC);
snapshotTimeHisto = metricsRegistry.newStat(
SNAPSHOT_TIME_NAME, SNAPSHOT_TIME_DESC, "Ops", "Time", true);
snapshotCloneTimeHisto = metricsRegistry.newStat(
SNAPSHOT_CLONE_TIME_NAME, SNAPSHOT_CLONE_TIME_DESC, "Ops", "Time", true);
snapshotRestoreTimeHisto = metricsRegistry.newStat(
SNAPSHOT_RESTORE_TIME_NAME, SNAPSHOT_RESTORE_TIME_DESC, "Ops", "Time", true);
}
public void incRequests(final int inc) {
@ -97,6 +107,21 @@ public class MetricsMasterSourceImpl
splitSizeHisto.add(size);
}
@Override
public void updateSnapshotTime(long time) {
snapshotTimeHisto.add(time);
}
@Override
public void updateSnapshotCloneTime(long time) {
snapshotCloneTimeHisto.add(time);
}
@Override
public void updateSnapshotRestoreTime(long time) {
snapshotRestoreTimeHisto.add(time);
}
@Override
public void getMetrics(MetricsCollector metricsCollector, boolean all) {

View File

@ -618,7 +618,7 @@ Server {
", cluster-up flag was=" + wasUp);
// create the snapshot manager
this.snapshotManager = new SnapshotManager(this);
this.snapshotManager = new SnapshotManager(this, this.metricsMaster);
}
/**

View File

@ -89,4 +89,28 @@ public class MetricsMaster {
public void updateRITOldestAge(long timestamp) {
masterSource.setRITOldestAge(timestamp);
}
/**
* Record a single instance of a snapshot
* @param time time that the snapshot took
*/
public void addSnapshot(long time) {
masterSource.updateSnapshotTime(time);
}
/**
* Record a single instance of a snapshot
* @param time time that the snapshot restore took
*/
public void addSnapshotRestore(long time) {
masterSource.updateSnapshotRestoreTime(time);
}
/**
* Record a single instance of a snapshot cloned table
* @param time time that the snapshot clone took
*/
public void addSnapshotClone(long time) {
masterSource.updateSnapshotCloneTime(time);
}
}

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.exceptions.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.exceptions.RestoreSnapshotException;
import org.apache.hadoop.hbase.exceptions.TableExistsException;
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.master.handler.CreateTableHandler;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@ -62,15 +63,18 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
private final SnapshotDescription snapshot;
private final ForeignExceptionDispatcher monitor;
private final MetricsMaster metricsMaster;
private final MonitoredTask status;
private volatile boolean stopped = false;
public CloneSnapshotHandler(final MasterServices masterServices,
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor,
final MetricsMaster metricsMaster)
throws NotAllMetaRegionsOnlineException, TableExistsException, IOException {
super(masterServices, masterServices.getMasterFileSystem(), hTableDescriptor,
masterServices.getConfiguration(), null, masterServices);
this.metricsMaster = metricsMaster;
// Snapshot information
this.snapshot = snapshot;
@ -141,6 +145,7 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
} else {
status.markComplete("Snapshot '"+ snapshot.getName() +"' clone completed and table enabled!");
}
metricsMaster.addSnapshotClone(status.getCompletionTimestamp() - status.getStartTime());
super.completed(exception);
}

View File

@ -32,6 +32,7 @@ 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.master.MetricsMaster;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
@ -64,8 +65,8 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
* @throws IOException on unexpected error
*/
public DisabledTableSnapshotHandler(SnapshotDescription snapshot,
final MasterServices masterServices) throws IOException {
super(snapshot, masterServices);
final MasterServices masterServices, final MetricsMaster metricsMaster) throws IOException {
super(snapshot, masterServices, metricsMaster);
// setup the timer
timeoutInjector = TakeSnapshotUtils.getMasterTimerAndBindToMonitor(snapshot, conf, monitor);

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.exceptions.HBaseSnapshotException;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.MetricsMaster;
import org.apache.hadoop.hbase.procedure.Procedure;
import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
@ -49,8 +50,8 @@ public class EnabledTableSnapshotHandler extends TakeSnapshotHandler {
private final ProcedureCoordinator coordinator;
public EnabledTableSnapshotHandler(SnapshotDescription snapshot, MasterServices master,
SnapshotManager manager) throws IOException {
super(snapshot, master);
final SnapshotManager manager, final MetricsMaster metricsMaster) throws IOException {
super(snapshot, master, metricsMaster);
this.coordinator = manager.getCoordinator();
}

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.MasterFileSystem;
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.master.handler.TableEventHandler;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@ -62,14 +63,16 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
private final SnapshotDescription snapshot;
private final ForeignExceptionDispatcher monitor;
private final MetricsMaster metricsMaster;
private final MonitoredTask status;
private volatile boolean stopped = false;
public RestoreSnapshotHandler(final MasterServices masterServices,
final SnapshotDescription snapshot, final HTableDescriptor htd)
throws IOException {
final SnapshotDescription snapshot, final HTableDescriptor htd,
final MetricsMaster metricsMaster) throws IOException {
super(EventType.C_M_RESTORE_SNAPSHOT, htd.getName(), masterServices, masterServices);
this.metricsMaster = metricsMaster;
// Snapshot information
this.snapshot = snapshot;
@ -146,6 +149,8 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
} else {
status.markComplete("Restore snapshot '"+ snapshot.getName() +"'!");
}
metricsMaster.addSnapshotRestore(status.getCompletionTimestamp() - status.getStartTime());
super.completed(exception);
}
@Override

View File

@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
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.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner;
@ -118,6 +119,7 @@ public class SnapshotManager implements Stoppable {
private boolean stopped;
private final long wakeFrequency;
private final MasterServices master; // Needed by TableEventHandlers
private final MetricsMaster metricsMaster;
private final ProcedureCoordinator coordinator;
// Is snapshot feature enabled?
@ -139,9 +141,11 @@ public class SnapshotManager implements Stoppable {
* Construct a snapshot manager.
* @param master
*/
public SnapshotManager(final MasterServices master) throws KeeperException, IOException,
UnsupportedOperationException {
public SnapshotManager(final MasterServices master, final MetricsMaster metricsMaster)
throws KeeperException, IOException, UnsupportedOperationException {
this.master = master;
this.metricsMaster = metricsMaster;
checkSnapshotSupport(master.getConfiguration(), master.getMasterFileSystem());
// get the configuration for the coordinator
@ -166,9 +170,12 @@ public class SnapshotManager implements Stoppable {
* @param coordinator procedure coordinator instance. exposed for testing.
* @param pool HBase ExecutorServcie instance, exposed for testing.
*/
public SnapshotManager(final MasterServices master, ProcedureCoordinator coordinator, ExecutorService pool)
public SnapshotManager(final MasterServices master, final MetricsMaster metricsMaster,
ProcedureCoordinator coordinator, ExecutorService pool)
throws IOException, UnsupportedOperationException {
this.master = master;
this.metricsMaster = metricsMaster;
checkSnapshotSupport(master.getConfiguration(), master.getMasterFileSystem());
this.wakeFrequency = master.getConfiguration().getInt(SNAPSHOT_WAKE_MILLIS_KEY,
@ -428,7 +435,7 @@ public class SnapshotManager implements Stoppable {
throws HBaseSnapshotException {
TakeSnapshotHandler handler;
try {
handler = new EnabledTableSnapshotHandler(snapshot, master, this).prepare();
handler = new EnabledTableSnapshotHandler(snapshot, master, this, metricsMaster).prepare();
this.executorService.submit(handler);
this.handler = handler;
} catch (Exception e) {
@ -537,7 +544,7 @@ public class SnapshotManager implements Stoppable {
DisabledTableSnapshotHandler handler;
try {
handler = new DisabledTableSnapshotHandler(snapshot, this.master).prepare();
handler = new DisabledTableSnapshotHandler(snapshot, master, metricsMaster).prepare();
this.executorService.submit(handler);
this.handler = handler;
} catch (Exception e) {
@ -620,7 +627,7 @@ public class SnapshotManager implements Stoppable {
try {
CloneSnapshotHandler handler =
new CloneSnapshotHandler(master, snapshot, hTableDescriptor).prepare();
new CloneSnapshotHandler(master, snapshot, hTableDescriptor, metricsMaster).prepare();
this.executorService.submit(handler);
restoreHandlers.put(tableName, handler);
} catch (Exception e) {
@ -711,7 +718,7 @@ public class SnapshotManager implements Stoppable {
try {
RestoreSnapshotHandler handler =
new RestoreSnapshotHandler(master, snapshot, hTableDescriptor);
new RestoreSnapshotHandler(master, snapshot, hTableDescriptor, metricsMaster);
this.executorService.submit(handler);
restoreHandlers.put(hTableDescriptor.getNameAsString(), handler);
} catch (Exception e) {

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.exceptions.SnapshotCreationException;
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.MetricsMaster;
import org.apache.hadoop.hbase.master.SnapshotSentinel;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
@ -70,6 +71,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
// none of these should ever be null
protected final MasterServices master;
protected final MetricsMaster metricsMaster;
protected final SnapshotDescription snapshot;
protected final Configuration conf;
protected final FileSystem fs;
@ -87,13 +89,14 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
* @param masterServices master services provider
* @throws IOException on unexpected error
*/
public TakeSnapshotHandler(SnapshotDescription snapshot,
final MasterServices masterServices) throws IOException {
public TakeSnapshotHandler(SnapshotDescription snapshot, final MasterServices masterServices,
final MetricsMaster metricsMaster) throws IOException {
super(masterServices, EventType.C_M_SNAPSHOT_TABLE);
assert snapshot != null : "SnapshotDescription must not be nul1";
assert masterServices != null : "MasterServices must not be nul1";
this.master = masterServices;
this.metricsMaster = metricsMaster;
this.snapshot = snapshot;
this.conf = this.master.getConfiguration();
this.fs = this.master.getMasterFileSystem().getFileSystem();
@ -173,6 +176,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
completeSnapshot(this.snapshotDir, this.workingDir, this.fs);
status.markComplete("Snapshot " + snapshot.getName() + " of table " + snapshot.getTable()
+ " completed");
metricsMaster.addSnapshot(status.getCompletionTimestamp() - status.getStartTime());
} catch (Exception e) {
status.abort("Failed to complete snapshot " + snapshot.getName() + " on table " +
snapshot.getTable() + " because " + e.getMessage());

View File

@ -63,6 +63,187 @@ import org.apache.hadoop.hbase.util.FSTableDescriptors;
public final class SnapshotInfo extends Configured implements Tool {
private static final Log LOG = LogFactory.getLog(SnapshotInfo.class);
/**
* Statistics about the snapshot
* <ol>
* <li> How many store files and logs are in the archive
* <li> How many store files and logs are shared with the table
* <li> Total store files and logs size and shared amount
* </ol>
*/
public static class SnapshotStats {
/** Information about the file referenced by the snapshot */
static class FileInfo {
private final boolean inArchive;
private final long size;
FileInfo(final boolean inArchive, final long size) {
this.inArchive = inArchive;
this.size = size;
}
/** @return true if the file is in the archive */
public boolean inArchive() {
return this.inArchive;
}
/** @return true if the file is missing */
public boolean isMissing() {
return this.size < 0;
}
/** @return the file size */
public long getSize() {
return this.size;
}
}
private int hfileArchiveCount = 0;
private int hfilesMissing = 0;
private int hfilesCount = 0;
private int logsMissing = 0;
private int logsCount = 0;
private long hfileArchiveSize = 0;
private long hfileSize = 0;
private long logSize = 0;
private final SnapshotDescription snapshot;
private final Configuration conf;
private final FileSystem fs;
SnapshotStats(final Configuration conf, final FileSystem fs, final SnapshotDescription snapshot)
{
this.snapshot = snapshot;
this.conf = conf;
this.fs = fs;
}
/** @return the snapshot descriptor */
public SnapshotDescription getSnapshotDescription() {
return this.snapshot;
}
/** @return true if the snapshot is corrupted */
public boolean isSnapshotCorrupted() {
return hfilesMissing > 0 || logsMissing > 0;
}
/** @return the number of available store files */
public int getStoreFilesCount() {
return hfilesCount + hfileArchiveCount;
}
/** @return the number of available store files in the archive */
public int getArchivedStoreFilesCount() {
return hfileArchiveCount;
}
/** @return the number of available log files */
public int getLogsCount() {
return logsCount;
}
/** @return the number of missing store files */
public int getMissingStoreFilesCount() {
return hfilesMissing;
}
/** @return the number of missing log files */
public int getMissingLogsCount() {
return logsMissing;
}
/** @return the total size of the store files referenced by the snapshot */
public long getStoreFilesSize() {
return hfileSize + hfileArchiveSize;
}
/** @return the total size of the store files shared */
public long getSharedStoreFilesSize() {
return hfileSize;
}
/** @return the total size of the store files in the archive */
public long getArchivedStoreFileSize() {
return hfileArchiveSize;
}
/** @return the percentage of the shared store files */
public float getSharedStoreFilePercentage() {
return ((float)hfileSize / (hfileSize + hfileArchiveSize)) * 100;
}
/** @return the total log size */
public long getLogsSize() {
return logSize;
}
/**
* Add the specified store file to the stats
* @param region region encoded Name
* @param family family name
* @param hfile store file name
* @return the store file information
*/
FileInfo addStoreFile(final String region, final String family, final String hfile)
throws IOException {
String table = this.snapshot.getTable();
Path path = new Path(family, HFileLink.createHFileLinkName(table, region, hfile));
HFileLink link = new HFileLink(conf, path);
boolean inArchive = false;
long size = -1;
try {
if ((inArchive = fs.exists(link.getArchivePath()))) {
size = fs.getFileStatus(link.getArchivePath()).getLen();
hfileArchiveSize += size;
hfileArchiveCount++;
} else {
size = link.getFileStatus(fs).getLen();
hfileSize += size;
hfilesCount++;
}
} catch (FileNotFoundException e) {
hfilesMissing++;
}
return new FileInfo(inArchive, size);
}
/**
* Add the specified recovered.edits file to the stats
* @param region region encoded name
* @param logfile log file name
* @return the recovered.edits information
*/
FileInfo addRecoveredEdits(final String region, final String logfile) throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
Path path = SnapshotReferenceUtil.getRecoveredEdits(snapshotDir, region, logfile);
long size = fs.getFileStatus(path).getLen();
logSize += size;
logsCount++;
return new FileInfo(true, size);
}
/**
* Add the specified log file to the stats
* @param server server name
* @param logfile log file name
* @return the log information
*/
FileInfo addLogFile(final String server, final String logfile) throws IOException {
HLogLink logLink = new HLogLink(conf, server, logfile);
long size = -1;
try {
size = logLink.getFileStatus(fs).getLen();
logSize += size;
logsCount++;
} catch (FileNotFoundException e) {
logsMissing++;
}
return new FileInfo(false, size);
}
}
private FileSystem fs;
private Path rootDir;
@ -170,104 +351,68 @@ public final class SnapshotInfo extends Configured implements Tool {
* dump the file list if requested and the collected information.
*/
private void printFiles(final boolean showFiles) throws IOException {
final String table = snapshotDesc.getTable();
final Configuration conf = getConf();
if (showFiles) {
System.out.println("Snapshot Files");
System.out.println("----------------------------------------");
}
// Collect information about hfiles and logs in the snapshot
final AtomicInteger hfileArchiveCount = new AtomicInteger();
final AtomicInteger hfilesMissing = new AtomicInteger();
final AtomicInteger hfilesCount = new AtomicInteger();
final AtomicInteger logsMissing = new AtomicInteger();
final AtomicInteger logsCount = new AtomicInteger();
final AtomicLong hfileArchiveSize = new AtomicLong();
final AtomicLong hfileSize = new AtomicLong();
final AtomicLong logSize = new AtomicLong();
final String table = this.snapshotDesc.getTable();
final SnapshotStats stats = new SnapshotStats(this.getConf(), this.fs, this.snapshotDesc);
SnapshotReferenceUtil.visitReferencedFiles(fs, snapshotDir,
new SnapshotReferenceUtil.FileVisitor() {
public void storeFile (final String region, final String family, final String hfile)
throws IOException {
Path path = new Path(family, HFileLink.createHFileLinkName(table, region, hfile));
HFileLink link = new HFileLink(conf, path);
boolean inArchive = false;
long size = -1;
try {
if ((inArchive = fs.exists(link.getArchivePath()))) {
size = fs.getFileStatus(link.getArchivePath()).getLen();
hfileArchiveSize.addAndGet(size);
hfileArchiveCount.addAndGet(1);
} else {
size = link.getFileStatus(fs).getLen();
hfileSize.addAndGet(size);
hfilesCount.addAndGet(1);
}
} catch (FileNotFoundException e) {
hfilesMissing.addAndGet(1);
}
SnapshotStats.FileInfo info = stats.addStoreFile(region, family, hfile);
if (showFiles) {
System.out.printf("%8s %s/%s/%s/%s %s%n",
(size < 0 ? "-" : StringUtils.humanReadableInt(size)),
(info.isMissing() ? "-" : StringUtils.humanReadableInt(info.getSize())),
table, region, family, hfile,
(inArchive ? "(archive)" : (size < 0) ? "(NOT FOUND)" : ""));
(info.inArchive() ? "(archive)" : info.isMissing() ? "(NOT FOUND)" : ""));
}
}
public void recoveredEdits (final String region, final String logfile)
throws IOException {
Path path = SnapshotReferenceUtil.getRecoveredEdits(snapshotDir, region, logfile);
long size = fs.getFileStatus(path).getLen();
logSize.addAndGet(size);
logsCount.addAndGet(1);
SnapshotStats.FileInfo info = stats.addRecoveredEdits(region, logfile);
if (showFiles) {
System.out.printf("%8s recovered.edits %s on region %s%n",
StringUtils.humanReadableInt(size), logfile, region);
StringUtils.humanReadableInt(info.getSize()), logfile, region);
}
}
public void logFile (final String server, final String logfile)
throws IOException {
HLogLink logLink = new HLogLink(conf, server, logfile);
long size = -1;
try {
size = logLink.getFileStatus(fs).getLen();
logSize.addAndGet(size);
logsCount.addAndGet(1);
} catch (FileNotFoundException e) {
logsMissing.addAndGet(1);
}
SnapshotStats.FileInfo info = stats.addLogFile(server, logfile);
if (showFiles) {
System.out.printf("%8s log %s on server %s %s%n",
(size < 0 ? "-" : StringUtils.humanReadableInt(size)),
(info.isMissing() ? "-" : StringUtils.humanReadableInt(info.getSize())),
logfile, server,
(size < 0 ? "(NOT FOUND)" : ""));
(info.isMissing() ? "(NOT FOUND)" : ""));
}
}
});
// Dump the stats
System.out.println();
if (hfilesMissing.get() > 0 || logsMissing.get() > 0) {
if (stats.isSnapshotCorrupted()) {
System.out.println("**************************************************************");
System.out.printf("BAD SNAPSHOT: %d hfile(s) and %d log(s) missing.%n",
hfilesMissing.get(), logsMissing.get());
stats.getMissingStoreFilesCount(), stats.getMissingLogsCount());
System.out.println("**************************************************************");
}
System.out.printf("%d HFiles (%d in archive), total size %s (%.2f%% %s shared with the source table)%n",
hfilesCount.get() + hfileArchiveCount.get(), hfileArchiveCount.get(),
StringUtils.humanReadableInt(hfileSize.get() + hfileArchiveSize.get()),
((float)hfileSize.get() / (hfileSize.get() + hfileArchiveSize.get())) * 100,
StringUtils.humanReadableInt(hfileSize.get())
stats.getStoreFilesCount(), stats.getArchivedStoreFilesCount(),
StringUtils.humanReadableInt(stats.getStoreFilesSize()),
stats.getSharedStoreFilePercentage(),
StringUtils.humanReadableInt(stats.getSharedStoreFilesSize())
);
System.out.printf("%d Logs, total size %s%n",
logsCount.get(), StringUtils.humanReadableInt(logSize.get()));
stats.getLogsCount(), StringUtils.humanReadableInt(stats.getLogsSize()));
System.out.println();
}
@ -286,6 +431,36 @@ public final class SnapshotInfo extends Configured implements Tool {
System.exit(1);
}
/**
* Returns the snapshot stats
* @param conf the {@link Configuration} to use
* @param snapshot {@link SnapshotDescription} to get stats from
* @return the snapshot stats
*/
public static SnapshotStats getSnapshotStats(final Configuration conf,
final SnapshotDescription snapshot) throws IOException {
Path rootDir = FSUtils.getRootDir(conf);
FileSystem fs = FileSystem.get(conf);
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
final SnapshotStats stats = new SnapshotStats(conf, fs, snapshot);
SnapshotReferenceUtil.visitReferencedFiles(fs, snapshotDir,
new SnapshotReferenceUtil.FileVisitor() {
public void storeFile (final String region, final String family, final String hfile)
throws IOException {
stats.addStoreFile(region, family, hfile);
}
public void recoveredEdits (final String region, final String logfile) throws IOException {
stats.addRecoveredEdits(region, logfile);
}
public void logFile (final String server, final String logfile) throws IOException {
stats.addLogFile(server, logfile);
}
});
return stats;
}
/**
* The guts of the {@link #main} method.
* Call this method to avoid the {@link #main(String[])} System.exit.

View File

@ -28,10 +28,12 @@
import="org.apache.hadoop.hbase.ServerLoad"
import="org.apache.hadoop.hbase.RegionLoad"
import="org.apache.hadoop.hbase.master.HMaster"
import="org.apache.hadoop.hbase.snapshot.SnapshotInfo"
import="org.apache.hadoop.hbase.util.Bytes"
import="org.apache.hadoop.hbase.util.FSUtils"
import="org.apache.hadoop.hbase.protobuf.ProtobufUtil"
import="org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription"
import="org.apache.hadoop.util.StringUtils"
import="java.util.List"
import="java.util.Map"
import="org.apache.hadoop.hbase.HConstants"%><%
@ -41,9 +43,11 @@
boolean readOnly = conf.getBoolean("hbase.master.ui.readonly", false);
String snapshotName = request.getParameter("name");
SnapshotDescription snapshot = null;
SnapshotInfo.SnapshotStats stats = null;
for (SnapshotDescription snapshotDesc: hbadmin.listSnapshots()) {
if (snapshotName.equals(snapshotDesc.getName())) {
snapshot = snapshotDesc;
stats = SnapshotInfo.getSnapshotStats(conf, snapshot);
break;
}
}
@ -158,14 +162,44 @@
<th>Creation Time</th>
<th>Type</th>
<th>Format Version</th>
<th>State</th>
</tr>
<tr>
<td><a href="table.jsp?name=<%= snapshot.getTable() %>"><%= snapshot.getTable() %></a></td>
<td><%= new Date(snapshot.getCreationTime()) %></td>
<td><%= snapshot.getType() %></td>
<td><%= snapshot.getVersion() %></td>
<% if (stats.isSnapshotCorrupted()) { %>
<td style="font-weight: bold; color: #dd0000;">CORRUPTED</td>
<% } else { %>
<td>ok</td>
<% } %>
</tr>
</table>
<div class="row">
<div class="span12">
<%= stats.getStoreFilesCount() %> HFiles (<%= stats.getArchivedStoreFilesCount() %> in archive),
total size <%= StringUtils.humanReadableInt(stats.getStoreFilesSize()) %>
(<%= stats.getSharedStoreFilePercentage() %>&#37;
<%= StringUtils.humanReadableInt(stats.getSharedStoreFilesSize()) %> shared with the source
table)
</div>
<div class="span12">
<%= stats.getLogsCount() %> Logs, total size
<%= StringUtils.humanReadableInt(stats.getLogsSize()) %>
</div>
</div>
<% if (stats.isSnapshotCorrupted()) { %>
<div class="row">
<div class="span12">
<h3>CORRUPTED Snapshot</h3>
</div>
<div class="span12">
<%= stats.getMissingStoreFilesCount() %> hfile(s) and
<%= stats.getMissingLogsCount() %> log(s) missing.
</div>
</div>
<% } %>
<%
} // end else

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.MetricsMaster;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner;
import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
@ -49,6 +50,7 @@ public class TestSnapshotManager {
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
MasterServices services = Mockito.mock(MasterServices.class);
MetricsMaster metrics = Mockito.mock(MetricsMaster.class);
ProcedureCoordinator coordinator = Mockito.mock(ProcedureCoordinator.class);
ExecutorService pool = Mockito.mock(ExecutorService.class);
MasterFileSystem mfs = Mockito.mock(MasterFileSystem.class);
@ -71,7 +73,7 @@ public class TestSnapshotManager {
Mockito.when(services.getMasterFileSystem()).thenReturn(mfs);
Mockito.when(mfs.getFileSystem()).thenReturn(fs);
Mockito.when(mfs.getRootDir()).thenReturn(UTIL.getDataTestDir());
return new SnapshotManager(services, coordinator, pool);
return new SnapshotManager(services, metrics, coordinator, pool);
}
@Test