|
@@ -52,6 +52,7 @@ import org.apache.hadoop.hdfs.util.AtomicFileOutputStream;
|
|
|
import org.apache.hadoop.hdfs.util.BestEffortLongFile;
|
|
|
import org.apache.hadoop.hdfs.util.PersistentLongFile;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
+import org.apache.hadoop.ipc.Server;
|
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
@@ -87,6 +88,17 @@ class Journal implements Closeable {
|
|
|
* number of that writer is stored persistently on disk.
|
|
|
*/
|
|
|
private PersistentLongFile lastPromisedEpoch;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Each IPC that comes from a given client contains a serial number
|
|
|
+ * which only increases from the client's perspective. Whenever
|
|
|
+ * we switch epochs, we reset this back to -1. Whenever an IPC
|
|
|
+ * comes from a client, we ensure that it is strictly higher
|
|
|
+ * than any previous IPC. This guards against any bugs in the IPC
|
|
|
+ * layer that would re-order IPCs or cause a stale retry from an old
|
|
|
+ * request to resurface and confuse things.
|
|
|
+ */
|
|
|
+ private long currentEpochIpcSerial = -1;
|
|
|
|
|
|
/**
|
|
|
* The epoch number of the last writer to actually write a transaction.
|
|
@@ -262,13 +274,15 @@ class Journal implements Closeable {
|
|
|
|
|
|
checkFormatted();
|
|
|
storage.checkConsistentNamespace(nsInfo);
|
|
|
-
|
|
|
+
|
|
|
+ // Check that the new epoch being proposed is in fact newer than
|
|
|
+ // any other that we've promised.
|
|
|
if (epoch <= getLastPromisedEpoch()) {
|
|
|
throw new IOException("Proposed epoch " + epoch + " <= last promise " +
|
|
|
getLastPromisedEpoch());
|
|
|
}
|
|
|
|
|
|
- lastPromisedEpoch.set(epoch);
|
|
|
+ updateLastPromisedEpoch(epoch);
|
|
|
abortCurSegment();
|
|
|
|
|
|
NewEpochResponseProto.Builder builder =
|
|
@@ -283,6 +297,16 @@ class Journal implements Closeable {
|
|
|
return builder.build();
|
|
|
}
|
|
|
|
|
|
+ private void updateLastPromisedEpoch(long newEpoch) throws IOException {
|
|
|
+ LOG.info("Updating lastPromisedEpoch from " + lastPromisedEpoch.get() +
|
|
|
+ " to " + newEpoch + " for client " + Server.getRemoteIp());
|
|
|
+ lastPromisedEpoch.set(newEpoch);
|
|
|
+
|
|
|
+ // Since we have a new writer, reset the IPC serial - it will start
|
|
|
+ // counting again from 0 for this writer.
|
|
|
+ currentEpochIpcSerial = -1;
|
|
|
+ }
|
|
|
+
|
|
|
private void abortCurSegment() throws IOException {
|
|
|
if (curSegment == null) {
|
|
|
return;
|
|
@@ -372,14 +396,19 @@ class Journal implements Closeable {
|
|
|
throw new IOException("IPC's epoch " + reqInfo.getEpoch() +
|
|
|
" is less than the last promised epoch " +
|
|
|
lastPromisedEpoch.get());
|
|
|
+ } else if (reqInfo.getEpoch() > lastPromisedEpoch.get()) {
|
|
|
+ // A newer client has arrived. Fence any previous writers by updating
|
|
|
+ // the promise.
|
|
|
+ updateLastPromisedEpoch(reqInfo.getEpoch());
|
|
|
}
|
|
|
|
|
|
- // TODO: should other requests check the _exact_ epoch instead of
|
|
|
- // the <= check? <= should probably only be necessary for the
|
|
|
- // first calls
|
|
|
-
|
|
|
- // TODO: some check on serial number that they only increase from a given
|
|
|
- // client
|
|
|
+ // Ensure that the IPCs are arriving in-order as expected.
|
|
|
+ checkSync(reqInfo.getIpcSerialNumber() > currentEpochIpcSerial,
|
|
|
+ "IPC serial %s from client %s was not higher than prior highest " +
|
|
|
+ "IPC serial %s", reqInfo.getIpcSerialNumber(),
|
|
|
+ Server.getRemoteIp(),
|
|
|
+ currentEpochIpcSerial);
|
|
|
+ currentEpochIpcSerial = reqInfo.getIpcSerialNumber();
|
|
|
|
|
|
if (reqInfo.hasCommittedTxId()) {
|
|
|
Preconditions.checkArgument(
|
|
@@ -424,6 +453,22 @@ class Journal implements Closeable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * @throws AssertionError if the given expression is not true.
|
|
|
+ * The message of the exception is formatted using the 'msg' and
|
|
|
+ * 'formatArgs' parameters.
|
|
|
+ *
|
|
|
+ * This should be used in preference to Java's built-in assert in
|
|
|
+ * non-performance-critical paths, where a failure of this invariant
|
|
|
+ * might cause the protocol to lose data.
|
|
|
+ */
|
|
|
+ private void alwaysAssert(boolean expression, String msg,
|
|
|
+ Object... formatArgs) {
|
|
|
+ if (!expression) {
|
|
|
+ throw new AssertionError(String.format(msg, formatArgs));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Start a new segment at the given txid. The previous segment
|
|
|
* must have already been finalized.
|
|
@@ -466,7 +511,9 @@ class Journal implements Closeable {
|
|
|
|
|
|
long curLastWriterEpoch = lastWriterEpoch.get();
|
|
|
if (curLastWriterEpoch != reqInfo.getEpoch()) {
|
|
|
- LOG.info("Recording lastWriterEpoch = " + reqInfo.getEpoch());
|
|
|
+ LOG.info("Updating lastWriterEpoch from " + curLastWriterEpoch +
|
|
|
+ " to " + reqInfo.getEpoch() + " for client " +
|
|
|
+ Server.getRemoteIp());
|
|
|
lastWriterEpoch.set(reqInfo.getEpoch());
|
|
|
}
|
|
|
|
|
@@ -689,9 +736,8 @@ class Journal implements Closeable {
|
|
|
|
|
|
long segmentTxId = segment.getStartTxId();
|
|
|
|
|
|
- // TODO: right now, a recovery of a segment when the log is
|
|
|
- // completely emtpy (ie startLogSegment() but no txns)
|
|
|
- // will fail this assertion here, since endTxId < startTxId
|
|
|
+ // Basic sanity checks that the segment is well-formed and contains
|
|
|
+ // at least one transaction.
|
|
|
Preconditions.checkArgument(segment.getEndTxId() > 0 &&
|
|
|
segment.getEndTxId() >= segmentTxId,
|
|
|
"bad recovery state for segment %s: %s",
|
|
@@ -702,8 +748,12 @@ class Journal implements Closeable {
|
|
|
.setAcceptedInEpoch(reqInfo.getEpoch())
|
|
|
.setSegmentState(segment)
|
|
|
.build();
|
|
|
+
|
|
|
+ // If we previously acted on acceptRecovery() from a higher-numbered writer,
|
|
|
+ // this call is out of sync. We should never actually trigger this, since the
|
|
|
+ // checkRequest() call above should filter non-increasing epoch numbers.
|
|
|
if (oldData != null) {
|
|
|
- Preconditions.checkState(oldData.getAcceptedInEpoch() <= reqInfo.getEpoch(),
|
|
|
+ alwaysAssert(oldData.getAcceptedInEpoch() <= reqInfo.getEpoch(),
|
|
|
"Bad paxos transition, out-of-order epochs.\nOld: %s\nNew: %s\n",
|
|
|
oldData, newData);
|
|
|
}
|
|
@@ -737,6 +787,12 @@ class Journal implements Closeable {
|
|
|
committedTxnId.get());
|
|
|
}
|
|
|
|
|
|
+ // Another paranoid check: we should not be asked to synchronize a log
|
|
|
+ // on top of a finalized segment.
|
|
|
+ alwaysAssert(currentSegment.getIsInProgress(),
|
|
|
+ "Should never be asked to synchronize a different log on top of an " +
|
|
|
+ "already-finalized segment");
|
|
|
+
|
|
|
// If we're shortening the log, update our highest txid
|
|
|
// used for lag metrics.
|
|
|
if (txnRange(currentSegment).contains(highestWrittenTxId)) {
|