|
@@ -46,10 +46,13 @@ import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
|
|
|
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 com.google.common.base.Preconditions;
|
|
|
+import com.google.common.collect.Range;
|
|
|
+import com.google.common.collect.Ranges;
|
|
|
import com.google.protobuf.ByteString;
|
|
|
import com.google.protobuf.TextFormat;
|
|
|
|
|
@@ -85,9 +88,18 @@ class Journal implements Closeable {
|
|
|
*/
|
|
|
private PersistentLongFile lastWriterEpoch;
|
|
|
|
|
|
+ /**
|
|
|
+ * Lower-bound on the last committed transaction ID. This is not
|
|
|
+ * depended upon for correctness, but acts as a sanity check
|
|
|
+ * during the recovery procedures, and as a visibility mark
|
|
|
+ * for clients reading in-progress logs.
|
|
|
+ */
|
|
|
+ private BestEffortLongFile committedTxnId;
|
|
|
+
|
|
|
private static final String LAST_PROMISED_FILENAME = "last-promised-epoch";
|
|
|
private static final String LAST_WRITER_EPOCH = "last-writer-epoch";
|
|
|
-
|
|
|
+ private static final String COMMITTED_TXID_FILENAME = "committed-txid";
|
|
|
+
|
|
|
private final FileJournalManager fjm;
|
|
|
|
|
|
Journal(File logDir, StorageErrorReporter errorReporter) throws IOException {
|
|
@@ -98,7 +110,10 @@ class Journal implements Closeable {
|
|
|
new File(currentDir, LAST_PROMISED_FILENAME), 0);
|
|
|
this.lastWriterEpoch = new PersistentLongFile(
|
|
|
new File(currentDir, LAST_WRITER_EPOCH), 0);
|
|
|
-
|
|
|
+ this.committedTxnId = new BestEffortLongFile(
|
|
|
+ new File(currentDir, COMMITTED_TXID_FILENAME),
|
|
|
+ HdfsConstants.INVALID_TXID);
|
|
|
+
|
|
|
this.fjm = storage.getJournalManager();
|
|
|
}
|
|
|
|
|
@@ -113,22 +128,21 @@ class Journal implements Closeable {
|
|
|
}
|
|
|
LOG.info("Scanning storage " + fjm);
|
|
|
List<EditLogFile> files = fjm.getLogFiles(0);
|
|
|
- if (files.isEmpty()) {
|
|
|
- curSegmentTxId = HdfsConstants.INVALID_TXID;
|
|
|
- return;
|
|
|
- }
|
|
|
+ curSegmentTxId = HdfsConstants.INVALID_TXID;
|
|
|
|
|
|
- EditLogFile latestLog = files.get(files.size() - 1);
|
|
|
- latestLog.validateLog();
|
|
|
- LOG.info("Latest log is " + latestLog);
|
|
|
- if (latestLog.getLastTxId() == HdfsConstants.INVALID_TXID) {
|
|
|
- // the log contains no transactions
|
|
|
- LOG.warn("Latest log " + latestLog + " has no transactions. " +
|
|
|
- "moving it aside");
|
|
|
- latestLog.moveAsideEmptyFile();
|
|
|
- curSegmentTxId = HdfsConstants.INVALID_TXID;
|
|
|
- } else {
|
|
|
- curSegmentTxId = latestLog.getFirstTxId();
|
|
|
+ while (!files.isEmpty()) {
|
|
|
+ EditLogFile latestLog = files.remove(files.size() - 1);
|
|
|
+ latestLog.validateLog();
|
|
|
+ LOG.info("Latest log is " + latestLog);
|
|
|
+ if (latestLog.getLastTxId() == HdfsConstants.INVALID_TXID) {
|
|
|
+ // the log contains no transactions
|
|
|
+ LOG.warn("Latest log " + latestLog + " has no transactions. " +
|
|
|
+ "moving it aside and looking for previous log");
|
|
|
+ latestLog.moveAsideEmptyFile();
|
|
|
+ } else {
|
|
|
+ curSegmentTxId = latestLog.getFirstTxId();
|
|
|
+ break;
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -150,6 +164,8 @@ class Journal implements Closeable {
|
|
|
@Override // Closeable
|
|
|
public void close() throws IOException {
|
|
|
storage.close();
|
|
|
+
|
|
|
+ IOUtils.closeStream(committedTxnId);
|
|
|
}
|
|
|
|
|
|
JNStorage getStorage() {
|
|
@@ -164,6 +180,10 @@ class Journal implements Closeable {
|
|
|
checkFormatted();
|
|
|
return lastPromisedEpoch.get();
|
|
|
}
|
|
|
+
|
|
|
+ synchronized long getCommittedTxnIdForTests() throws IOException {
|
|
|
+ return committedTxnId.get();
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* Try to create a new epoch for this journal.
|
|
@@ -213,8 +233,8 @@ class Journal implements Closeable {
|
|
|
synchronized void journal(RequestInfo reqInfo,
|
|
|
long segmentTxId, long firstTxnId,
|
|
|
int numTxns, byte[] records) throws IOException {
|
|
|
- checkWriteRequest(reqInfo);
|
|
|
checkFormatted();
|
|
|
+ checkWriteRequest(reqInfo);
|
|
|
|
|
|
// TODO: if a JN goes down and comes back up, then it will throw
|
|
|
// this exception on every edit. We should instead send back
|
|
@@ -245,6 +265,7 @@ class Journal implements Closeable {
|
|
|
if (LOG.isTraceEnabled()) {
|
|
|
LOG.trace("Writing txid " + firstTxnId + "-" + (firstTxnId + numTxns - 1));
|
|
|
}
|
|
|
+
|
|
|
curSegment.writeRaw(records, 0, records.length);
|
|
|
curSegment.setReadyToFlush();
|
|
|
curSegment.flush();
|
|
@@ -270,6 +291,15 @@ class Journal implements Closeable {
|
|
|
|
|
|
// TODO: some check on serial number that they only increase from a given
|
|
|
// client
|
|
|
+
|
|
|
+ if (reqInfo.hasCommittedTxId()) {
|
|
|
+ Preconditions.checkArgument(
|
|
|
+ reqInfo.getCommittedTxId() >= committedTxnId.get(),
|
|
|
+ "Client trying to move committed txid backward from " +
|
|
|
+ committedTxnId.get() + " to " + reqInfo.getCommittedTxId());
|
|
|
+
|
|
|
+ committedTxnId.set(reqInfo.getCommittedTxId());
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private synchronized void checkWriteRequest(RequestInfo reqInfo) throws IOException {
|
|
@@ -296,8 +326,8 @@ class Journal implements Closeable {
|
|
|
public synchronized void startLogSegment(RequestInfo reqInfo, long txid)
|
|
|
throws IOException {
|
|
|
assert fjm != null;
|
|
|
- checkRequest(reqInfo);
|
|
|
checkFormatted();
|
|
|
+ checkRequest(reqInfo);
|
|
|
|
|
|
if (curSegment != null) {
|
|
|
LOG.warn("Client is requesting a new log segment " + txid +
|
|
@@ -352,8 +382,8 @@ class Journal implements Closeable {
|
|
|
*/
|
|
|
public synchronized void finalizeLogSegment(RequestInfo reqInfo, long startTxId,
|
|
|
long endTxId) throws IOException {
|
|
|
- checkRequest(reqInfo);
|
|
|
checkFormatted();
|
|
|
+ checkRequest(reqInfo);
|
|
|
|
|
|
if (startTxId == curSegmentTxId) {
|
|
|
if (curSegment != null) {
|
|
@@ -397,8 +427,8 @@ class Journal implements Closeable {
|
|
|
*/
|
|
|
public synchronized void purgeLogsOlderThan(RequestInfo reqInfo,
|
|
|
long minTxIdToKeep) throws IOException {
|
|
|
- checkRequest(reqInfo);
|
|
|
checkFormatted();
|
|
|
+ checkRequest(reqInfo);
|
|
|
|
|
|
fjm.purgeLogsOlderThan(minTxIdToKeep);
|
|
|
purgePaxosDecisionsOlderThan(minTxIdToKeep);
|
|
@@ -492,8 +522,8 @@ class Journal implements Closeable {
|
|
|
*/
|
|
|
public synchronized PrepareRecoveryResponseProto prepareRecovery(
|
|
|
RequestInfo reqInfo, long segmentTxId) throws IOException {
|
|
|
- checkRequest(reqInfo);
|
|
|
checkFormatted();
|
|
|
+ checkRequest(reqInfo);
|
|
|
|
|
|
PrepareRecoveryResponseProto.Builder builder =
|
|
|
PrepareRecoveryResponseProto.newBuilder();
|
|
@@ -519,6 +549,9 @@ class Journal implements Closeable {
|
|
|
}
|
|
|
|
|
|
builder.setLastWriterEpoch(lastWriterEpoch.get());
|
|
|
+ if (committedTxnId.get() != HdfsConstants.INVALID_TXID) {
|
|
|
+ builder.setLastCommittedTxId(committedTxnId.get());
|
|
|
+ }
|
|
|
|
|
|
PrepareRecoveryResponseProto resp = builder.build();
|
|
|
LOG.info("Prepared recovery for segment " + segmentTxId + ": " +
|
|
@@ -532,8 +565,8 @@ class Journal implements Closeable {
|
|
|
public synchronized void acceptRecovery(RequestInfo reqInfo,
|
|
|
SegmentStateProto segment, URL fromUrl)
|
|
|
throws IOException {
|
|
|
- checkRequest(reqInfo);
|
|
|
checkFormatted();
|
|
|
+ checkRequest(reqInfo);
|
|
|
long segmentTxId = segment.getStartTxId();
|
|
|
|
|
|
// TODO: right now, a recovery of a segment when the log is
|
|
@@ -563,8 +596,22 @@ class Journal implements Closeable {
|
|
|
": no current segment in place");
|
|
|
} else {
|
|
|
LOG.info("Synchronizing log " + TextFormat.shortDebugString(segment) +
|
|
|
- ": old segment " + TextFormat.shortDebugString(segment) + " is " +
|
|
|
- "not the right length");
|
|
|
+ ": old segment " + TextFormat.shortDebugString(currentSegment) +
|
|
|
+ " is not the right length");
|
|
|
+
|
|
|
+ // Paranoid sanity check: if the new log is shorter than the log we
|
|
|
+ // currently have, we should not end up discarding any transactions
|
|
|
+ // which are already Committed.
|
|
|
+ if (txnRange(currentSegment).contains(committedTxnId.get()) &&
|
|
|
+ !txnRange(segment).contains(committedTxnId.get())) {
|
|
|
+ throw new AssertionError(
|
|
|
+ "Cannot replace segment " +
|
|
|
+ TextFormat.shortDebugString(currentSegment) +
|
|
|
+ " with new segment " +
|
|
|
+ TextFormat.shortDebugString(segment) +
|
|
|
+ ": would discard already-committed txn " +
|
|
|
+ committedTxnId.get());
|
|
|
+ }
|
|
|
}
|
|
|
syncLog(reqInfo, segment, fromUrl);
|
|
|
} else {
|
|
@@ -581,6 +628,12 @@ class Journal implements Closeable {
|
|
|
TextFormat.shortDebugString(newData));
|
|
|
}
|
|
|
|
|
|
+ private Range<Long> txnRange(SegmentStateProto seg) {
|
|
|
+ Preconditions.checkArgument(seg.hasEndTxId(),
|
|
|
+ "invalid segment: %s", seg);
|
|
|
+ return Ranges.closed(seg.getStartTxId(), seg.getEndTxId());
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Synchronize a log segment from another JournalNode.
|
|
|
* @param reqInfo the request info for the recovery IPC
|