|
@@ -25,13 +25,13 @@ import java.util.concurrent.Callable;
|
|
|
import java.util.concurrent.ExecutionException;
|
|
|
import java.util.concurrent.ExecutorService;
|
|
|
import java.util.concurrent.Executors;
|
|
|
-import java.util.concurrent.ScheduledExecutorService;
|
|
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
|
|
+import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException;
|
|
|
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
|
|
|
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
|
|
|
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
|
|
@@ -98,6 +98,13 @@ public class IPCLoggerChannel implements AsyncLogger {
|
|
|
*/
|
|
|
private final int queueSizeLimitBytes;
|
|
|
|
|
|
+ /**
|
|
|
+ * If this logger misses some edits, or restarts in the middle of
|
|
|
+ * a segment, the writer won't be able to write any more edits until
|
|
|
+ * the beginning of the next segment. Upon detecting this situation,
|
|
|
+ * the writer sets this flag to true to avoid sending useless RPCs.
|
|
|
+ */
|
|
|
+ private boolean outOfSync = false;
|
|
|
|
|
|
static final Factory FACTORY = new AsyncLogger.Factory() {
|
|
|
@Override
|
|
@@ -212,6 +219,15 @@ public class IPCLoggerChannel implements AsyncLogger {
|
|
|
public synchronized int getQueuedEditsSize() {
|
|
|
return queuedEditsSizeBytes;
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * @return true if the server has gotten out of sync from the client,
|
|
|
+ * and thus a log roll is required for this logger to successfully start
|
|
|
+ * logging more edits.
|
|
|
+ */
|
|
|
+ public synchronized boolean isOutOfSync() {
|
|
|
+ return outOfSync;
|
|
|
+ }
|
|
|
|
|
|
@VisibleForTesting
|
|
|
void waitForAllPendingCalls() throws InterruptedException {
|
|
@@ -265,8 +281,22 @@ public class IPCLoggerChannel implements AsyncLogger {
|
|
|
ret = executor.submit(new Callable<Void>() {
|
|
|
@Override
|
|
|
public Void call() throws IOException {
|
|
|
- getProxy().journal(createReqInfo(),
|
|
|
- segmentTxId, firstTxnId, numTxns, data);
|
|
|
+ throwIfOutOfSync();
|
|
|
+
|
|
|
+ try {
|
|
|
+ getProxy().journal(createReqInfo(),
|
|
|
+ segmentTxId, firstTxnId, numTxns, data);
|
|
|
+ } catch (IOException e) {
|
|
|
+ QuorumJournalManager.LOG.warn(
|
|
|
+ "Remote journal " + IPCLoggerChannel.this + " failed to " +
|
|
|
+ "write txns " + firstTxnId + "-" + (firstTxnId + numTxns - 1) +
|
|
|
+ ". Will try to write to this JN again after the next " +
|
|
|
+ "log roll.", e);
|
|
|
+ synchronized (IPCLoggerChannel.this) {
|
|
|
+ outOfSync = true;
|
|
|
+ }
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
synchronized (IPCLoggerChannel.this) {
|
|
|
highestAckedTxId = firstTxnId + numTxns - 1;
|
|
|
}
|
|
@@ -298,6 +328,15 @@ 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
|
|
|
+ throw new JournalOutOfSyncException(
|
|
|
+ "Journal disabled until next roll");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private synchronized void reserveQueueSpace(int size)
|
|
|
throws LoggerTooFarBehindException {
|
|
|
Preconditions.checkArgument(size >= 0);
|
|
@@ -330,6 +369,15 @@ public class IPCLoggerChannel implements AsyncLogger {
|
|
|
@Override
|
|
|
public Void call() throws IOException {
|
|
|
getProxy().startLogSegment(createReqInfo(), txid);
|
|
|
+ synchronized (IPCLoggerChannel.this) {
|
|
|
+ if (outOfSync) {
|
|
|
+ outOfSync = false;
|
|
|
+ QuorumJournalManager.LOG.info(
|
|
|
+ "Restarting previously-stopped writes to " +
|
|
|
+ IPCLoggerChannel.this + " in segment starting at txid " +
|
|
|
+ txid);
|
|
|
+ }
|
|
|
+ }
|
|
|
return null;
|
|
|
}
|
|
|
});
|
|
@@ -341,6 +389,8 @@ public class IPCLoggerChannel implements AsyncLogger {
|
|
|
return executor.submit(new Callable<Void>() {
|
|
|
@Override
|
|
|
public Void call() throws IOException {
|
|
|
+ throwIfOutOfSync();
|
|
|
+
|
|
|
getProxy().finalizeLogSegment(createReqInfo(), startTxId, endTxId);
|
|
|
return null;
|
|
|
}
|
|
@@ -415,5 +465,8 @@ public class IPCLoggerChannel implements AsyncLogger {
|
|
|
if (behind > 0) {
|
|
|
sb.append(" (" + behind + " behind)");
|
|
|
}
|
|
|
+ if (outOfSync) {
|
|
|
+ sb.append(" (will re-join on next segment)");
|
|
|
+ }
|
|
|
}
|
|
|
-}
|
|
|
+}
|