|
@@ -36,6 +36,7 @@ import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
import org.apache.hadoop.thirdparty.com.google.common.collect.Iterators;
|
|
|
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
+import org.apache.hadoop.util.Timer;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
@@ -55,12 +56,10 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
|
|
|
import org.apache.hadoop.ipc.RPC;
|
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
|
|
|
|
-import static org.apache.hadoop.util.Time.monotonicNow;
|
|
|
import static org.apache.hadoop.util.ExitUtil.terminate;
|
|
|
|
|
|
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
|
|
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
|
|
-import org.apache.hadoop.util.Time;
|
|
|
|
|
|
|
|
|
/**
|
|
@@ -172,14 +171,21 @@ public class EditLogTailer {
|
|
|
*/
|
|
|
private final long maxTxnsPerLock;
|
|
|
|
|
|
+ /**
|
|
|
+ * Timer instance to be set only using constructor.
|
|
|
+ * Only tests can reassign this by using setTimerForTests().
|
|
|
+ * For source code, this timer instance should be treated as final.
|
|
|
+ */
|
|
|
+ private Timer timer;
|
|
|
+
|
|
|
public EditLogTailer(FSNamesystem namesystem, Configuration conf) {
|
|
|
this.tailerThread = new EditLogTailerThread();
|
|
|
this.conf = conf;
|
|
|
this.namesystem = namesystem;
|
|
|
+ this.timer = new Timer();
|
|
|
this.editLog = namesystem.getEditLog();
|
|
|
-
|
|
|
- lastLoadTimeMs = monotonicNow();
|
|
|
- lastRollTimeMs = monotonicNow();
|
|
|
+ this.lastLoadTimeMs = timer.monotonicNow();
|
|
|
+ this.lastRollTimeMs = timer.monotonicNow();
|
|
|
|
|
|
logRollPeriodMs = conf.getTimeDuration(
|
|
|
DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY,
|
|
@@ -301,7 +307,7 @@ public class EditLogTailer {
|
|
|
long editsTailed = 0;
|
|
|
// Fully tail the journal to the end
|
|
|
do {
|
|
|
- long startTime = Time.monotonicNow();
|
|
|
+ long startTime = timer.monotonicNow();
|
|
|
try {
|
|
|
NameNode.getNameNodeMetrics().addEditLogTailInterval(
|
|
|
startTime - lastLoadTimeMs);
|
|
@@ -312,7 +318,7 @@ public class EditLogTailer {
|
|
|
throw new IOException(e);
|
|
|
} finally {
|
|
|
NameNode.getNameNodeMetrics().addEditLogTailTime(
|
|
|
- Time.monotonicNow() - startTime);
|
|
|
+ timer.monotonicNow() - startTime);
|
|
|
}
|
|
|
} while(editsTailed > 0);
|
|
|
return null;
|
|
@@ -336,7 +342,7 @@ public class EditLogTailer {
|
|
|
LOG.debug("lastTxnId: " + lastTxnId);
|
|
|
}
|
|
|
Collection<EditLogInputStream> streams;
|
|
|
- long startTime = Time.monotonicNow();
|
|
|
+ long startTime = timer.monotonicNow();
|
|
|
try {
|
|
|
streams = editLog.selectInputStreams(lastTxnId + 1, 0,
|
|
|
null, inProgressOk, true);
|
|
@@ -349,7 +355,7 @@ public class EditLogTailer {
|
|
|
return 0;
|
|
|
} finally {
|
|
|
NameNode.getNameNodeMetrics().addEditLogFetchTime(
|
|
|
- Time.monotonicNow() - startTime);
|
|
|
+ timer.monotonicNow() - startTime);
|
|
|
}
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("edit streams to load from: " + streams.size());
|
|
@@ -374,7 +380,7 @@ public class EditLogTailer {
|
|
|
}
|
|
|
|
|
|
if (editsLoaded > 0) {
|
|
|
- lastLoadTimeMs = monotonicNow();
|
|
|
+ lastLoadTimeMs = timer.monotonicNow();
|
|
|
}
|
|
|
lastLoadedTxnId = image.getLastAppliedTxId();
|
|
|
return editsLoaded;
|
|
@@ -395,7 +401,7 @@ public class EditLogTailer {
|
|
|
*/
|
|
|
private boolean tooLongSinceLastLoad() {
|
|
|
return logRollPeriodMs >= 0 &&
|
|
|
- (monotonicNow() - lastRollTimeMs) > logRollPeriodMs;
|
|
|
+ (timer.monotonicNow() - lastRollTimeMs) > logRollPeriodMs;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -423,9 +429,9 @@ public class EditLogTailer {
|
|
|
try {
|
|
|
future = rollEditsRpcExecutor.submit(getNameNodeProxy());
|
|
|
future.get(rollEditsTimeoutMs, TimeUnit.MILLISECONDS);
|
|
|
- lastRollTimeMs = monotonicNow();
|
|
|
+ this.lastRollTimeMs = timer.monotonicNow();
|
|
|
lastRollTriggerTxId = lastLoadedTxnId;
|
|
|
- } catch (ExecutionException e) {
|
|
|
+ } catch (ExecutionException | InterruptedException e) {
|
|
|
LOG.warn("Unable to trigger a roll of the active NN", e);
|
|
|
} catch (TimeoutException e) {
|
|
|
if (future != null) {
|
|
@@ -433,11 +439,30 @@ public class EditLogTailer {
|
|
|
}
|
|
|
LOG.warn(String.format(
|
|
|
"Unable to finish rolling edits in %d ms", rollEditsTimeoutMs));
|
|
|
- } catch (InterruptedException e) {
|
|
|
- LOG.warn("Unable to trigger a roll of the active NN", e);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * This is only to be used by tests. For source code, the only way to
|
|
|
+ * set timer is by using EditLogTailer constructor.
|
|
|
+ *
|
|
|
+ * @param newTimer Timer instance provided by tests.
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ void setTimerForTest(final Timer newTimer) {
|
|
|
+ this.timer = newTimer;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Used by tests. Return Timer instance used by EditLogTailer.
|
|
|
+ *
|
|
|
+ * @return Return Timer instance used by EditLogTailer.
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ Timer getTimer() {
|
|
|
+ return timer;
|
|
|
+ }
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
void sleep(long sleepTimeMillis) throws InterruptedException {
|
|
|
Thread.sleep(sleepTimeMillis);
|
|
@@ -497,7 +522,7 @@ public class EditLogTailer {
|
|
|
// name system lock will be acquired to further block even the block
|
|
|
// state updates.
|
|
|
namesystem.cpLockInterruptibly();
|
|
|
- long startTime = Time.monotonicNow();
|
|
|
+ long startTime = timer.monotonicNow();
|
|
|
try {
|
|
|
NameNode.getNameNodeMetrics().addEditLogTailInterval(
|
|
|
startTime - lastLoadTimeMs);
|
|
@@ -505,7 +530,7 @@ public class EditLogTailer {
|
|
|
} finally {
|
|
|
namesystem.cpUnlock();
|
|
|
NameNode.getNameNodeMetrics().addEditLogTailTime(
|
|
|
- Time.monotonicNow() - startTime);
|
|
|
+ timer.monotonicNow() - startTime);
|
|
|
}
|
|
|
//Update NameDirSize Metric
|
|
|
if (triggeredLogRoll) {
|