You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bh...@apache.org on 2020/07/17 20:27:25 UTC
[hbase] branch branch-1 updated: HBASE-24740 Enable journal logging
for HBase snapshot operation (#2076)
This is an automated email from the ASF dual-hosted git repository.
bharathv pushed a commit to branch branch-1
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-1 by this push:
new 5211859 HBASE-24740 Enable journal logging for HBase snapshot operation (#2076)
5211859 is described below
commit 5211859058ecd194461650dd811e3fe4708cbec4
Author: sguggilam <sa...@gmail.com>
AuthorDate: Fri Jul 17 13:27:08 2020 -0700
HBASE-24740 Enable journal logging for HBase snapshot operation (#2076)
Signed-off-by: Bharath Vissapragada <bh...@apache.org>
Signed-off-by: Mingliang Liu <li...@apache.org>
Signed-off-by: Viraj Jasani <vj...@apache.org>
---
.../hbase/master/snapshot/SnapshotManager.java | 33 ++++++++++++++--------
.../hbase/master/snapshot/TakeSnapshotHandler.java | 15 ++++++----
.../hadoop/hbase/monitoring/MonitoredTaskImpl.java | 6 ++--
.../hadoop/hbase/snapshot/SnapshotManifest.java | 31 ++++++++++++++------
4 files changed, 56 insertions(+), 29 deletions(-)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
index 7d25112..98018f0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
@@ -17,6 +17,9 @@
*/
package org.apache.hadoop.hbase.master.snapshot;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
@@ -34,22 +37,21 @@ import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.ipc.RpcServer;
@@ -89,9 +91,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.zookeeper.KeeperException;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
/**
* This class manages the procedure of taking and restoring snapshots. There is only one
* SnapshotManager for the master.
@@ -624,16 +623,26 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
AssignmentManager assignmentMgr = master.getAssignmentManager();
if (assignmentMgr.getTableStateManager().isTableState(snapshotTable,
ZooKeeperProtos.Table.State.ENABLED)) {
- LOG.debug("Table enabled, starting distributed snapshot.");
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Table enabled, starting distributed snapshot 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 (assignmentMgr.getTableStateManager().isTableState(snapshotTable,
ZooKeeperProtos.Table.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 "
+ + 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!");
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
index 3c8cb91..4ee07cb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
@@ -24,18 +24,17 @@ import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CancellationException;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
@@ -122,7 +121,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.tableLockManager = master.getTableLockManager();
this.tableLock = this.tableLockManager.writeLock(
@@ -134,6 +132,10 @@ 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 HTableDescriptor loadTableDescriptor()
@@ -243,6 +245,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());
+ }
releaseTableLock();
}
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java
index 41705db..4ef214d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java
@@ -18,15 +18,13 @@
*/
package org.apache.hadoop.hbase.monitoring;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.GsonUtil;
import org.apache.hbase.thirdparty.com.google.gson.Gson;
@@ -258,7 +256,7 @@ class MonitoredTaskImpl implements MonitoredTask {
if (journal == null) {
journal = new ArrayList<StatusJournalEntry>();
}
- if (includeCurrentStatus) {
+ if (includeCurrentStatus && status != null) {
journal.add(new StatusJournalEntryImpl(status, statusTime));
}
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
index 8a73773..4834333 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.snapshot;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.CodedInputStream;
import com.google.protobuf.InvalidProtocolBufferException;
-
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
@@ -31,7 +30,6 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
-
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -43,18 +41,19 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
+import org.apache.hadoop.hbase.monitoring.MonitoredTask;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest;
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
+import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
/**
@@ -82,6 +81,7 @@ public final class SnapshotManifest {
private final FileSystem rootFs;
private final FileSystem workingDirFs;
private int manifestSizeLimit;
+ private final MonitoredTask statusTask;
/**
*
@@ -95,12 +95,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);
}
@@ -121,10 +122,16 @@ 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);
+ }
+
/**
* Return a SnapshotManifest instance with the information already loaded in-memory.
* SnapshotManifest manifest = SnapshotManifest.open(...)
@@ -136,7 +143,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;
}
@@ -371,6 +378,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
@@ -395,7 +408,7 @@ public final class SnapshotManifest {
new FSTableDescriptors(conf, workingDirFs, rootDir)
.createTableDescriptorForTableDirectory(workingDir, htd, false);
} else {
- LOG.debug("Convert to Single Snapshot Manifest");
+ LOG.debug("Convert to Single Snapshot Manifest for " + this.desc.getName());
convertToV2SingleManifest();
}
}
@@ -408,6 +421,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);
@@ -431,6 +445,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();