HBASE-24740 Enable journal logging for HBase snapshot operation (#2104)

Signed-off-by: Bharath Vissapragada <bharathv@apache.org>
This commit is contained in:
sguggilam 2020-07-20 18:11:41 -07:00 committed by GitHub
parent 6cf013ddc4
commit 430602aed8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 43 additions and 13 deletions

View File

@ -654,16 +654,26 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
TableName snapshotTable = TableName.valueOf(snapshot.getTable());
if (master.getTableStateManager().isTableState(snapshotTable,
TableState.State.ENABLED)) {
LOG.debug("Table enabled, starting distributed snapshot.");
if (LOG.isDebugEnabled()) {
LOG.debug("Table enabled, starting distributed snapshots for {}",
ClientSnapshotDescriptionUtils.toString(snapshot));
}
snapshotEnabledTable(snapshot);
LOG.debug("Started snapshot: " + ClientSnapshotDescriptionUtils.toString(snapshot));
if (LOG.isDebugEnabled()) {
LOG.debug("Started snapshot: {}", ClientSnapshotDescriptionUtils.toString(snapshot));
}
}
// For disabled table, snapshot is created by the master
else if (master.getTableStateManager().isTableState(snapshotTable,
TableState.State.DISABLED)) {
LOG.debug("Table is disabled, running snapshot entirely on master.");
if (LOG.isDebugEnabled()) {
LOG.debug("Table is disabled, running snapshot entirely on master for {}",
ClientSnapshotDescriptionUtils.toString(snapshot));
}
snapshotDisabledTable(snapshot);
LOG.debug("Started snapshot: " + ClientSnapshotDescriptionUtils.toString(snapshot));
if (LOG.isDebugEnabled()) {
LOG.debug("Started snapshot: {}", ClientSnapshotDescriptionUtils.toString(snapshot));
}
} else {
LOG.error("Can't snapshot table '" + snapshot.getTable()
+ "', isn't open or closed, we don't know what to do!");

View File

@ -19,14 +19,12 @@ package org.apache.hadoop.hbase.master.snapshot;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.URI;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CancellationException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
@ -47,7 +45,6 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure2.LockType;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.CommonFSUtils;
@ -126,7 +123,6 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
this.snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
this.workingDirFs = this.workingDir.getFileSystem(this.conf);
this.monitor = new ForeignExceptionDispatcher(snapshot.getName());
this.snapshotManifest = SnapshotManifest.create(conf, rootFs, workingDir, snapshot, monitor);
this.tableLock = master.getLockManager().createMasterLock(
snapshotTable, LockType.EXCLUSIVE,
@ -137,6 +133,9 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
// update the running tasks
this.status = TaskMonitor.get().createStatus(
"Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable);
this.status.enableStatusJournal(true);
this.snapshotManifest =
SnapshotManifest.create(conf, rootFs, workingDir, snapshot, monitor, status);
}
private TableDescriptor loadTableDescriptor()
@ -256,6 +255,9 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
} catch (IOException e) {
LOG.error("Couldn't delete snapshot working directory:" + workingDir);
}
if (LOG.isDebugEnabled()) {
LOG.debug("Table snapshot journal : \n" + status.prettyPrintJournal());
}
tableLockToRelease.release();
}
}

View File

@ -259,7 +259,7 @@ class MonitoredTaskImpl implements MonitoredTask {
if (journal == null) {
journal = new ArrayList<StatusJournalEntry>();
}
if (includeCurrentStatus) {
if (includeCurrentStatus && status != null) {
journal.add(new StatusJournalEntryImpl(status, statusTime));
}
}

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.HStore;
@ -85,6 +86,7 @@ public final class SnapshotManifest {
private final FileSystem rootFs;
private final FileSystem workingDirFs;
private int manifestSizeLimit;
private final MonitoredTask statusTask;
/**
*
@ -98,12 +100,13 @@ public final class SnapshotManifest {
*/
private SnapshotManifest(final Configuration conf, final FileSystem rootFs,
final Path workingDir, final SnapshotDescription desc,
final ForeignExceptionSnare monitor) throws IOException {
final ForeignExceptionSnare monitor, final MonitoredTask statusTask) throws IOException {
this.monitor = monitor;
this.desc = desc;
this.workingDir = workingDir;
this.conf = conf;
this.rootFs = rootFs;
this.statusTask = statusTask;
this.workingDirFs = this.workingDir.getFileSystem(this.conf);
this.manifestSizeLimit = conf.getInt(SNAPSHOT_MANIFEST_SIZE_LIMIT_CONF_KEY, 64 * 1024 * 1024);
}
@ -124,7 +127,14 @@ public final class SnapshotManifest {
public static SnapshotManifest create(final Configuration conf, final FileSystem fs,
final Path workingDir, final SnapshotDescription desc,
final ForeignExceptionSnare monitor) throws IOException {
return new SnapshotManifest(conf, fs, workingDir, desc, monitor);
return create(conf, fs, workingDir, desc, monitor, null);
}
public static SnapshotManifest create(final Configuration conf, final FileSystem fs,
final Path workingDir, final SnapshotDescription desc, final ForeignExceptionSnare monitor,
final MonitoredTask statusTask) throws IOException {
return new SnapshotManifest(conf, fs, workingDir, desc, monitor, statusTask);
}
@ -139,7 +149,7 @@ public final class SnapshotManifest {
*/
public static SnapshotManifest open(final Configuration conf, final FileSystem fs,
final Path workingDir, final SnapshotDescription desc) throws IOException {
SnapshotManifest manifest = new SnapshotManifest(conf, fs, workingDir, desc, null);
SnapshotManifest manifest = new SnapshotManifest(conf, fs, workingDir, desc, null, null);
manifest.load();
return manifest;
}
@ -456,6 +466,12 @@ public final class SnapshotManifest {
return this.regionManifests;
}
private void setStatusMsg(String msg) {
if (this.statusTask != null) {
statusTask.setStatus(msg);
}
}
/**
* Get all the Region Manifest from the snapshot.
* This is an helper to get a map with the region encoded name
@ -478,7 +494,7 @@ public final class SnapshotManifest {
FSTableDescriptors.createTableDescriptorForTableDirectory(workingDirFs, workingDir, htd,
false);
} else {
LOG.debug("Convert to Single Snapshot Manifest");
LOG.debug("Convert to Single Snapshot Manifest for {}", this.desc.getName());
convertToV2SingleManifest();
}
}
@ -491,6 +507,7 @@ public final class SnapshotManifest {
// Try to load v1 and v2 regions
List<SnapshotRegionManifest> v1Regions, v2Regions;
ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader");
setStatusMsg("Loading Region manifests for " + this.desc.getName());
try {
v1Regions = SnapshotManifestV1.loadRegionManifests(conf, tpool, workingDirFs,
workingDir, desc);
@ -514,6 +531,7 @@ public final class SnapshotManifest {
// Once the data-manifest is written, the snapshot can be considered complete.
// Currently snapshots are written in a "temporary" directory and later
// moved to the "complated" snapshot directory.
setStatusMsg("Writing data manifest for " + this.desc.getName());
SnapshotDataManifest dataManifest = dataManifestBuilder.build();
writeDataManifest(dataManifest);
this.regionManifests = dataManifest.getRegionManifestsList();