|
@@ -26,6 +26,7 @@ import java.util.concurrent.Callable;
|
|
|
import java.util.concurrent.ExecutionException;
|
|
|
import java.util.concurrent.ExecutorService;
|
|
|
import java.util.concurrent.Executors;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -52,6 +53,7 @@ import org.apache.hadoop.security.SecurityUtil;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
+import com.google.common.base.Stopwatch;
|
|
|
import com.google.common.util.concurrent.FutureCallback;
|
|
|
import com.google.common.util.concurrent.Futures;
|
|
|
import com.google.common.util.concurrent.ListenableFuture;
|
|
@@ -92,6 +94,19 @@ public class IPCLoggerChannel implements AsyncLogger {
|
|
|
* The highest txid that has been successfully logged on the remote JN.
|
|
|
*/
|
|
|
private long highestAckedTxId = 0;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Nanotime of the last time we successfully journaled some edits
|
|
|
+ * to the remote node.
|
|
|
+ */
|
|
|
+ private long lastAckNanos = 0;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Nanotime of the last time that committedTxId was update. Used
|
|
|
+ * to calculate the lag in terms of time, rather than just a number
|
|
|
+ * of txns.
|
|
|
+ */
|
|
|
+ private long lastCommitNanos = 0;
|
|
|
|
|
|
/**
|
|
|
* The maximum number of bytes that can be pending in the queue.
|
|
@@ -109,6 +124,13 @@ public class IPCLoggerChannel implements AsyncLogger {
|
|
|
*/
|
|
|
private boolean outOfSync = false;
|
|
|
|
|
|
+ /**
|
|
|
+ * Stopwatch which starts counting on each heartbeat that is sent
|
|
|
+ */
|
|
|
+ private Stopwatch lastHeartbeatStopwatch = new Stopwatch();
|
|
|
+
|
|
|
+ private static final long HEARTBEAT_INTERVAL_MILLIS = 1000;
|
|
|
+
|
|
|
static final Factory FACTORY = new AsyncLogger.Factory() {
|
|
|
@Override
|
|
|
public AsyncLogger createLogger(Configuration conf, NamespaceInfo nsInfo,
|
|
@@ -145,6 +167,7 @@ public class IPCLoggerChannel implements AsyncLogger {
|
|
|
"Trying to move committed txid backwards in client " +
|
|
|
"old: %s new: %s", committedTxId, txid);
|
|
|
this.committedTxId = txid;
|
|
|
+ this.lastCommitNanos = System.nanoTime();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -295,6 +318,11 @@ public class IPCLoggerChannel implements AsyncLogger {
|
|
|
} catch (LoggerTooFarBehindException e) {
|
|
|
return Futures.immediateFailedFuture(e);
|
|
|
}
|
|
|
+
|
|
|
+ // When this batch is acked, we use its submission time in order
|
|
|
+ // to calculate how far we are lagging.
|
|
|
+ final long submitNanos = System.nanoTime();
|
|
|
+
|
|
|
ListenableFuture<Void> ret = null;
|
|
|
try {
|
|
|
ret = executor.submit(new Callable<Void>() {
|
|
@@ -318,6 +346,7 @@ public class IPCLoggerChannel implements AsyncLogger {
|
|
|
}
|
|
|
synchronized (IPCLoggerChannel.this) {
|
|
|
highestAckedTxId = firstTxnId + numTxns - 1;
|
|
|
+ lastAckNanos = submitNanos;
|
|
|
}
|
|
|
return null;
|
|
|
}
|
|
@@ -347,15 +376,40 @@ public class IPCLoggerChannel implements AsyncLogger {
|
|
|
return ret;
|
|
|
}
|
|
|
|
|
|
- private synchronized void throwIfOutOfSync() throws JournalOutOfSyncException {
|
|
|
- if (outOfSync) {
|
|
|
- // TODO: send a "heartbeat" here so that the remote node knows the newest
|
|
|
- // committed txid, for metrics purposes
|
|
|
+ private void throwIfOutOfSync()
|
|
|
+ throws JournalOutOfSyncException, IOException {
|
|
|
+ if (isOutOfSync()) {
|
|
|
+ // Even if we're out of sync, it's useful to send an RPC
|
|
|
+ // to the remote node in order to update its lag metrics, etc.
|
|
|
+ heartbeatIfNecessary();
|
|
|
throw new JournalOutOfSyncException(
|
|
|
"Journal disabled until next roll");
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * When we've entered an out-of-sync state, it's still useful to periodically
|
|
|
+ * send an empty RPC to the server, such that it has the up to date
|
|
|
+ * committedTxId. This acts as a sanity check during recovery, and also allows
|
|
|
+ * that node's metrics to be up-to-date about its lag.
|
|
|
+ *
|
|
|
+ * In the future, this method may also be used in order to check that the
|
|
|
+ * current node is still the current writer, even if no edits are being
|
|
|
+ * written.
|
|
|
+ */
|
|
|
+ private void heartbeatIfNecessary() throws IOException {
|
|
|
+ if (lastHeartbeatStopwatch.elapsedMillis() > HEARTBEAT_INTERVAL_MILLIS ||
|
|
|
+ !lastHeartbeatStopwatch.isRunning()) {
|
|
|
+ try {
|
|
|
+ getProxy().heartbeat(createReqInfo());
|
|
|
+ } finally {
|
|
|
+ // Don't send heartbeats more often than the configured interval,
|
|
|
+ // even if they fail.
|
|
|
+ lastHeartbeatStopwatch.reset().start();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private synchronized void reserveQueueSpace(int size)
|
|
|
throws LoggerTooFarBehindException {
|
|
|
Preconditions.checkArgument(size >= 0);
|
|
@@ -479,13 +533,27 @@ public class IPCLoggerChannel implements AsyncLogger {
|
|
|
@Override
|
|
|
public synchronized void appendHtmlReport(StringBuilder sb) {
|
|
|
sb.append("Written txid ").append(highestAckedTxId);
|
|
|
- long behind = committedTxId - highestAckedTxId;
|
|
|
- assert behind >= 0;
|
|
|
+ long behind = getLagTxns();
|
|
|
if (behind > 0) {
|
|
|
- sb.append(" (" + behind + " behind)");
|
|
|
+ if (lastAckNanos != 0) {
|
|
|
+ long lagMillis = getLagTimeMillis();
|
|
|
+ sb.append(" (" + behind + " txns/" + lagMillis + "ms behind)");
|
|
|
+ } else {
|
|
|
+ sb.append(" (never written");
|
|
|
+ }
|
|
|
}
|
|
|
if (outOfSync) {
|
|
|
- sb.append(" (will re-join on next segment)");
|
|
|
+ sb.append(" (will try to re-sync on next segment)");
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private long getLagTxns() {
|
|
|
+ return Math.max(committedTxId - highestAckedTxId, 0);
|
|
|
+ }
|
|
|
+
|
|
|
+ private long getLagTimeMillis() {
|
|
|
+ return TimeUnit.MILLISECONDS.convert(
|
|
|
+ Math.max(lastCommitNanos - lastAckNanos, 0),
|
|
|
+ TimeUnit.NANOSECONDS);
|
|
|
+ }
|
|
|
}
|