|
@@ -31,7 +31,6 @@ import java.util.regex.Pattern;
|
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.JournalManager.CorruptionException;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.StoragePurger;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
|
|
@@ -61,7 +60,6 @@ class FileJournalManager implements JournalManager {
|
|
|
NameNodeFile.EDITS_INPROGRESS.getName() + "_(\\d+)");
|
|
|
|
|
|
private File currentInProgress = null;
|
|
|
- private long maxSeenTransaction = 0L;
|
|
|
|
|
|
@VisibleForTesting
|
|
|
StoragePurger purger
|
|
@@ -143,7 +141,7 @@ class FileJournalManager implements JournalManager {
|
|
|
allLogFiles.size());
|
|
|
|
|
|
for (EditLogFile elf : allLogFiles) {
|
|
|
- if (elf.isCorrupt() || elf.isInProgress()) continue;
|
|
|
+ if (elf.hasCorruptHeader() || elf.isInProgress()) continue;
|
|
|
if (elf.getFirstTxId() >= firstTxId) {
|
|
|
ret.add(new RemoteEditLog(elf.firstTxId, elf.lastTxId));
|
|
|
} else if ((firstTxId > elf.getFirstTxId()) &&
|
|
@@ -244,7 +242,7 @@ class FileJournalManager implements JournalManager {
|
|
|
elf.validateLog();
|
|
|
}
|
|
|
|
|
|
- if (elf.isCorrupt()) {
|
|
|
+ if (elf.hasCorruptHeader()) {
|
|
|
break;
|
|
|
}
|
|
|
numTxns += elf.getLastTxId() + 1 - fromTxId;
|
|
@@ -281,22 +279,38 @@ class FileJournalManager implements JournalManager {
|
|
|
File currentDir = sd.getCurrentDir();
|
|
|
LOG.info("Recovering unfinalized segments in " + currentDir);
|
|
|
List<EditLogFile> allLogFiles = matchEditLogs(currentDir.listFiles());
|
|
|
-
|
|
|
- // make sure journal is aware of max seen transaction before moving corrupt
|
|
|
- // files aside
|
|
|
- findMaxTransaction(true);
|
|
|
|
|
|
for (EditLogFile elf : allLogFiles) {
|
|
|
if (elf.getFile().equals(currentInProgress)) {
|
|
|
continue;
|
|
|
}
|
|
|
if (elf.isInProgress()) {
|
|
|
+ // If the file is zero-length, we likely just crashed after opening the
|
|
|
+ // file, but before writing anything to it. Safe to delete it.
|
|
|
+ if (elf.getFile().length() == 0) {
|
|
|
+ LOG.info("Deleting zero-length edit log file " + elf);
|
|
|
+ elf.getFile().delete();
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+
|
|
|
elf.validateLog();
|
|
|
|
|
|
- if (elf.isCorrupt()) {
|
|
|
+ if (elf.hasCorruptHeader()) {
|
|
|
elf.moveAsideCorruptFile();
|
|
|
+ throw new CorruptionException("In-progress edit log file is corrupt: "
|
|
|
+ + elf);
|
|
|
+ }
|
|
|
+
|
|
|
+ // If the file has a valid header (isn't corrupt) but contains no
|
|
|
+ // transactions, we likely just crashed after opening the file and
|
|
|
+ // writing the header, but before syncing any transactions. Safe to
|
|
|
+ // delete the file.
|
|
|
+ if (elf.getNumTransactions() == 0) {
|
|
|
+ LOG.info("Deleting edit log file with zero transactions " + elf);
|
|
|
+ elf.getFile().delete();
|
|
|
continue;
|
|
|
}
|
|
|
+
|
|
|
finalizeLogSegment(elf.getFirstTxId(), elf.getLastTxId());
|
|
|
}
|
|
|
}
|
|
@@ -321,15 +335,21 @@ class FileJournalManager implements JournalManager {
|
|
|
|
|
|
/**
|
|
|
* Find the maximum transaction in the journal.
|
|
|
- * This gets stored in a member variable, as corrupt edit logs
|
|
|
- * will be moved aside, but we still need to remember their first
|
|
|
- * tranaction id in the case that it was the maximum transaction in
|
|
|
- * the journal.
|
|
|
*/
|
|
|
private long findMaxTransaction(boolean inProgressOk)
|
|
|
throws IOException {
|
|
|
+ boolean considerSeenTxId = true;
|
|
|
+ long seenTxId = NNStorage.readTransactionIdFile(sd);
|
|
|
+ long maxSeenTransaction = 0;
|
|
|
for (EditLogFile elf : getLogFiles(0)) {
|
|
|
if (elf.isInProgress() && !inProgressOk) {
|
|
|
+ if (elf.getFirstTxId() != HdfsConstants.INVALID_TXID &&
|
|
|
+ elf.getFirstTxId() <= seenTxId) {
|
|
|
+ // don't look at the seen_txid file if in-progress logs are not to be
|
|
|
+ // examined, and the value in seen_txid falls within the in-progress
|
|
|
+ // segment.
|
|
|
+ considerSeenTxId = false;
|
|
|
+ }
|
|
|
continue;
|
|
|
}
|
|
|
|
|
@@ -339,7 +359,11 @@ class FileJournalManager implements JournalManager {
|
|
|
}
|
|
|
maxSeenTransaction = Math.max(elf.getLastTxId(), maxSeenTransaction);
|
|
|
}
|
|
|
- return maxSeenTransaction;
|
|
|
+ if (considerSeenTxId) {
|
|
|
+ return Math.max(maxSeenTransaction, seenTxId);
|
|
|
+ } else {
|
|
|
+ return maxSeenTransaction;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -354,8 +378,9 @@ class FileJournalManager implements JournalManager {
|
|
|
private File file;
|
|
|
private final long firstTxId;
|
|
|
private long lastTxId;
|
|
|
+ private long numTx = -1;
|
|
|
|
|
|
- private boolean isCorrupt = false;
|
|
|
+ private boolean hasCorruptHeader = false;
|
|
|
private final boolean isInProgress;
|
|
|
|
|
|
final static Comparator<EditLogFile> COMPARE_BY_START_TXID
|
|
@@ -407,11 +432,13 @@ class FileJournalManager implements JournalManager {
|
|
|
*/
|
|
|
void validateLog() throws IOException {
|
|
|
EditLogValidation val = EditLogFileInputStream.validateEditLog(file);
|
|
|
- if (val.getNumTransactions() == 0) {
|
|
|
- markCorrupt();
|
|
|
- } else {
|
|
|
- this.lastTxId = val.getEndTxId();
|
|
|
- }
|
|
|
+ this.numTx = val.getNumTransactions();
|
|
|
+ this.lastTxId = val.getEndTxId();
|
|
|
+ this.hasCorruptHeader = val.hasCorruptHeader();
|
|
|
+ }
|
|
|
+
|
|
|
+ long getNumTransactions() {
|
|
|
+ return numTx;
|
|
|
}
|
|
|
|
|
|
boolean isInProgress() {
|
|
@@ -422,16 +449,12 @@ class FileJournalManager implements JournalManager {
|
|
|
return file;
|
|
|
}
|
|
|
|
|
|
- void markCorrupt() {
|
|
|
- isCorrupt = true;
|
|
|
- }
|
|
|
-
|
|
|
- boolean isCorrupt() {
|
|
|
- return isCorrupt;
|
|
|
+ boolean hasCorruptHeader() {
|
|
|
+ return hasCorruptHeader;
|
|
|
}
|
|
|
|
|
|
void moveAsideCorruptFile() throws IOException {
|
|
|
- assert isCorrupt;
|
|
|
+ assert hasCorruptHeader;
|
|
|
|
|
|
File src = file;
|
|
|
File dst = new File(src.getParent(), src.getName() + ".corrupt");
|
|
@@ -446,8 +469,9 @@ class FileJournalManager implements JournalManager {
|
|
|
@Override
|
|
|
public String toString() {
|
|
|
return String.format("EditLogFile(file=%s,first=%019d,last=%019d,"
|
|
|
- +"inProgress=%b,corrupt=%b)", file.toString(),
|
|
|
- firstTxId, lastTxId, isInProgress(), isCorrupt);
|
|
|
+ +"inProgress=%b,hasCorruptHeader=%b,numTx=%d)",
|
|
|
+ file.toString(), firstTxId, lastTxId,
|
|
|
+ isInProgress(), hasCorruptHeader, numTx);
|
|
|
}
|
|
|
}
|
|
|
}
|