|
@@ -18,7 +18,6 @@
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.server.namenode.FSImageFormat.renameReservedPathsOnUpgrade;
|
|
import static org.apache.hadoop.hdfs.server.namenode.FSImageFormat.renameReservedPathsOnUpgrade;
|
|
-import static org.apache.hadoop.util.Time.monotonicNow;
|
|
|
|
|
|
|
|
import java.io.FilterInputStream;
|
|
import java.io.FilterInputStream;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
@@ -113,11 +112,16 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
|
|
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
|
|
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
|
|
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
|
|
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
|
|
import org.apache.hadoop.hdfs.util.Holder;
|
|
import org.apache.hadoop.hdfs.util.Holder;
|
|
|
|
+import org.apache.hadoop.log.LogThrottlingHelper;
|
|
import org.apache.hadoop.util.ChunkedArrayList;
|
|
import org.apache.hadoop.util.ChunkedArrayList;
|
|
|
|
+import org.apache.hadoop.util.Timer;
|
|
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.base.Joiner;
|
|
import com.google.common.base.Joiner;
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
|
|
|
|
|
|
+import static org.apache.hadoop.log.LogThrottlingHelper.LogAction;
|
|
|
|
+
|
|
@InterfaceAudience.Private
|
|
@InterfaceAudience.Private
|
|
@InterfaceStability.Evolving
|
|
@InterfaceStability.Evolving
|
|
public class FSEditLogLoader {
|
|
public class FSEditLogLoader {
|
|
@@ -125,16 +129,29 @@ public class FSEditLogLoader {
|
|
LoggerFactory.getLogger(FSEditLogLoader.class.getName());
|
|
LoggerFactory.getLogger(FSEditLogLoader.class.getName());
|
|
static final long REPLAY_TRANSACTION_LOG_INTERVAL = 1000; // 1sec
|
|
static final long REPLAY_TRANSACTION_LOG_INTERVAL = 1000; // 1sec
|
|
|
|
|
|
|
|
+ /** Limit logging about edit loading to every 5 seconds max. */
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ static final long LOAD_EDIT_LOG_INTERVAL_MS = 5000;
|
|
|
|
+ private final LogThrottlingHelper loadEditsLogHelper =
|
|
|
|
+ new LogThrottlingHelper(LOAD_EDIT_LOG_INTERVAL_MS);
|
|
|
|
+
|
|
private final FSNamesystem fsNamesys;
|
|
private final FSNamesystem fsNamesys;
|
|
private final BlockManager blockManager;
|
|
private final BlockManager blockManager;
|
|
|
|
+ private final Timer timer;
|
|
private long lastAppliedTxId;
|
|
private long lastAppliedTxId;
|
|
/** Total number of end transactions loaded. */
|
|
/** Total number of end transactions loaded. */
|
|
private int totalEdits = 0;
|
|
private int totalEdits = 0;
|
|
|
|
|
|
public FSEditLogLoader(FSNamesystem fsNamesys, long lastAppliedTxId) {
|
|
public FSEditLogLoader(FSNamesystem fsNamesys, long lastAppliedTxId) {
|
|
|
|
+ this(fsNamesys, lastAppliedTxId, new Timer());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ FSEditLogLoader(FSNamesystem fsNamesys, long lastAppliedTxId, Timer timer) {
|
|
this.fsNamesys = fsNamesys;
|
|
this.fsNamesys = fsNamesys;
|
|
this.blockManager = fsNamesys.getBlockManager();
|
|
this.blockManager = fsNamesys.getBlockManager();
|
|
this.lastAppliedTxId = lastAppliedTxId;
|
|
this.lastAppliedTxId = lastAppliedTxId;
|
|
|
|
+ this.timer = timer;
|
|
}
|
|
}
|
|
|
|
|
|
long loadFSEdits(EditLogInputStream edits, long expectedStartingTxId)
|
|
long loadFSEdits(EditLogInputStream edits, long expectedStartingTxId)
|
|
@@ -155,14 +172,26 @@ public class FSEditLogLoader {
|
|
prog.beginStep(Phase.LOADING_EDITS, step);
|
|
prog.beginStep(Phase.LOADING_EDITS, step);
|
|
fsNamesys.writeLock();
|
|
fsNamesys.writeLock();
|
|
try {
|
|
try {
|
|
- long startTime = monotonicNow();
|
|
|
|
- FSImage.LOG.info("Start loading edits file " + edits.getName()
|
|
|
|
- + " maxTxnsToRead = " + maxTxnsToRead);
|
|
|
|
|
|
+ long startTime = timer.monotonicNow();
|
|
|
|
+ LogAction preLogAction = loadEditsLogHelper.record("pre", startTime);
|
|
|
|
+ if (preLogAction.shouldLog()) {
|
|
|
|
+ FSImage.LOG.info("Start loading edits file " + edits.getName()
|
|
|
|
+ + " maxTxnsToRead = " + maxTxnsToRead +
|
|
|
|
+ LogThrottlingHelper.getLogSupressionMessage(preLogAction));
|
|
|
|
+ }
|
|
long numEdits = loadEditRecords(edits, false, expectedStartingTxId,
|
|
long numEdits = loadEditRecords(edits, false, expectedStartingTxId,
|
|
maxTxnsToRead, startOpt, recovery);
|
|
maxTxnsToRead, startOpt, recovery);
|
|
- FSImage.LOG.info("Edits file " + edits.getName()
|
|
|
|
- + " of size " + edits.length() + " edits # " + numEdits
|
|
|
|
- + " loaded in " + (monotonicNow()-startTime)/1000 + " seconds");
|
|
|
|
|
|
+ long endTime = timer.monotonicNow();
|
|
|
|
+ LogAction postLogAction = loadEditsLogHelper.record("post", endTime,
|
|
|
|
+ numEdits, edits.length(), endTime - startTime);
|
|
|
|
+ if (postLogAction.shouldLog()) {
|
|
|
|
+ FSImage.LOG.info("Loaded {} edits file(s) (the last named {}) of " +
|
|
|
|
+ "total size {}, total edits {}, total load time {} ms",
|
|
|
|
+ postLogAction.getCount(), edits.getName(),
|
|
|
|
+ postLogAction.getStats(1).getSum(),
|
|
|
|
+ postLogAction.getStats(0).getSum(),
|
|
|
|
+ postLogAction.getStats(2).getSum());
|
|
|
|
+ }
|
|
return numEdits;
|
|
return numEdits;
|
|
} finally {
|
|
} finally {
|
|
edits.close();
|
|
edits.close();
|
|
@@ -203,7 +232,7 @@ public class FSEditLogLoader {
|
|
Step step = createStartupProgressStep(in);
|
|
Step step = createStartupProgressStep(in);
|
|
prog.setTotal(Phase.LOADING_EDITS, step, numTxns);
|
|
prog.setTotal(Phase.LOADING_EDITS, step, numTxns);
|
|
Counter counter = prog.getCounter(Phase.LOADING_EDITS, step);
|
|
Counter counter = prog.getCounter(Phase.LOADING_EDITS, step);
|
|
- long lastLogTime = monotonicNow();
|
|
|
|
|
|
+ long lastLogTime = timer.monotonicNow();
|
|
long lastInodeId = fsNamesys.dir.getLastInodeId();
|
|
long lastInodeId = fsNamesys.dir.getLastInodeId();
|
|
|
|
|
|
try {
|
|
try {
|
|
@@ -283,7 +312,7 @@ public class FSEditLogLoader {
|
|
}
|
|
}
|
|
// log progress
|
|
// log progress
|
|
if (op.hasTransactionId()) {
|
|
if (op.hasTransactionId()) {
|
|
- long now = monotonicNow();
|
|
|
|
|
|
+ long now = timer.monotonicNow();
|
|
if (now - lastLogTime > REPLAY_TRANSACTION_LOG_INTERVAL) {
|
|
if (now - lastLogTime > REPLAY_TRANSACTION_LOG_INTERVAL) {
|
|
long deltaTxId = lastAppliedTxId - expectedStartingTxId + 1;
|
|
long deltaTxId = lastAppliedTxId - expectedStartingTxId + 1;
|
|
int percent = Math.round((float) deltaTxId / numTxns * 100);
|
|
int percent = Math.round((float) deltaTxId / numTxns * 100);
|