|
@@ -37,9 +37,9 @@ import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.hdfs.protocol.FSConstants;
|
|
|
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
|
|
|
|
|
@@ -54,17 +54,13 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
private boolean needToSave = false;
|
|
|
private boolean isUpgradeFinalized = true;
|
|
|
|
|
|
- List<FoundFSImage> foundImages = new ArrayList<FoundFSImage>();
|
|
|
- List<FoundEditLog> foundEditLogs = new ArrayList<FoundEditLog>();
|
|
|
+ List<FSImageFile> foundImages = new ArrayList<FSImageFile>();
|
|
|
+ List<EditLogFile> foundEditLogs = new ArrayList<EditLogFile>();
|
|
|
SortedMap<Long, LogGroup> logGroups = new TreeMap<Long, LogGroup>();
|
|
|
long maxSeenTxId = 0;
|
|
|
|
|
|
private static final Pattern IMAGE_REGEX = Pattern.compile(
|
|
|
NameNodeFile.IMAGE.getName() + "_(\\d+)");
|
|
|
- private static final Pattern EDITS_REGEX = Pattern.compile(
|
|
|
- NameNodeFile.EDITS.getName() + "_(\\d+)-(\\d+)");
|
|
|
- private static final Pattern EDITS_INPROGRESS_REGEX = Pattern.compile(
|
|
|
- NameNodeFile.EDITS_INPROGRESS.getName() + "_(\\d+)");
|
|
|
|
|
|
@Override
|
|
|
public void inspectDirectory(StorageDirectory sd) throws IOException {
|
|
@@ -95,7 +91,7 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
if (sd.getStorageDirType().isOfType(NameNodeDirType.IMAGE)) {
|
|
|
try {
|
|
|
long txid = Long.valueOf(imageMatch.group(1));
|
|
|
- foundImages.add(new FoundFSImage(sd, f, txid));
|
|
|
+ foundImages.add(new FSImageFile(sd, f, txid));
|
|
|
} catch (NumberFormatException nfe) {
|
|
|
LOG.error("Image file " + f + " has improperly formatted " +
|
|
|
"transaction ID");
|
|
@@ -117,9 +113,10 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
LOG.warn("Unable to determine the max transaction ID seen by " + sd, ioe);
|
|
|
}
|
|
|
|
|
|
- List<FoundEditLog> editLogs = matchEditLogs(filesInStorage);
|
|
|
+ List<EditLogFile> editLogs
|
|
|
+ = FileJournalManager.matchEditLogs(filesInStorage);
|
|
|
if (sd.getStorageDirType().isOfType(NameNodeDirType.EDITS)) {
|
|
|
- for (FoundEditLog log : editLogs) {
|
|
|
+ for (EditLogFile log : editLogs) {
|
|
|
addEditLog(log);
|
|
|
}
|
|
|
} else if (!editLogs.isEmpty()){
|
|
@@ -133,47 +130,12 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
isUpgradeFinalized = isUpgradeFinalized && !sd.getPreviousDir().exists();
|
|
|
}
|
|
|
|
|
|
- static List<FoundEditLog> matchEditLogs(File[] filesInStorage) {
|
|
|
- List<FoundEditLog> ret = Lists.newArrayList();
|
|
|
- for (File f : filesInStorage) {
|
|
|
- String name = f.getName();
|
|
|
- // Check for edits
|
|
|
- Matcher editsMatch = EDITS_REGEX.matcher(name);
|
|
|
- if (editsMatch.matches()) {
|
|
|
- try {
|
|
|
- long startTxId = Long.valueOf(editsMatch.group(1));
|
|
|
- long endTxId = Long.valueOf(editsMatch.group(2));
|
|
|
- ret.add(new FoundEditLog(f, startTxId, endTxId));
|
|
|
- } catch (NumberFormatException nfe) {
|
|
|
- LOG.error("Edits file " + f + " has improperly formatted " +
|
|
|
- "transaction ID");
|
|
|
- // skip
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // Check for in-progress edits
|
|
|
- Matcher inProgressEditsMatch = EDITS_INPROGRESS_REGEX.matcher(name);
|
|
|
- if (inProgressEditsMatch.matches()) {
|
|
|
- try {
|
|
|
- long startTxId = Long.valueOf(inProgressEditsMatch.group(1));
|
|
|
- ret.add(
|
|
|
- new FoundEditLog(f, startTxId, FoundEditLog.UNKNOWN_END));
|
|
|
- } catch (NumberFormatException nfe) {
|
|
|
- LOG.error("In-progress edits file " + f + " has improperly " +
|
|
|
- "formatted transaction ID");
|
|
|
- // skip
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- return ret;
|
|
|
- }
|
|
|
-
|
|
|
- private void addEditLog(FoundEditLog foundEditLog) {
|
|
|
+ private void addEditLog(EditLogFile foundEditLog) {
|
|
|
foundEditLogs.add(foundEditLog);
|
|
|
- LogGroup group = logGroups.get(foundEditLog.startTxId);
|
|
|
+ LogGroup group = logGroups.get(foundEditLog.getFirstTxId());
|
|
|
if (group == null) {
|
|
|
- group = new LogGroup(foundEditLog.startTxId);
|
|
|
- logGroups.put(foundEditLog.startTxId, group);
|
|
|
+ group = new LogGroup(foundEditLog.getFirstTxId());
|
|
|
+ logGroups.put(foundEditLog.getFirstTxId(), group);
|
|
|
}
|
|
|
group.add(foundEditLog);
|
|
|
}
|
|
@@ -191,9 +153,9 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
*
|
|
|
* Returns null if no images were found.
|
|
|
*/
|
|
|
- FoundFSImage getLatestImage() {
|
|
|
- FoundFSImage ret = null;
|
|
|
- for (FoundFSImage img : foundImages) {
|
|
|
+ FSImageFile getLatestImage() {
|
|
|
+ FSImageFile ret = null;
|
|
|
+ for (FSImageFile img : foundImages) {
|
|
|
if (ret == null || img.txId > ret.txId) {
|
|
|
ret = img;
|
|
|
}
|
|
@@ -201,11 +163,11 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
return ret;
|
|
|
}
|
|
|
|
|
|
- public List<FoundFSImage> getFoundImages() {
|
|
|
+ public List<FSImageFile> getFoundImages() {
|
|
|
return ImmutableList.copyOf(foundImages);
|
|
|
}
|
|
|
|
|
|
- public List<FoundEditLog> getFoundEditLogs() {
|
|
|
+ public List<EditLogFile> getEditLogFiles() {
|
|
|
return ImmutableList.copyOf(foundEditLogs);
|
|
|
}
|
|
|
|
|
@@ -215,7 +177,7 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
throw new FileNotFoundException("No valid image files found");
|
|
|
}
|
|
|
|
|
|
- FoundFSImage recoveryImage = getLatestImage();
|
|
|
+ FSImageFile recoveryImage = getLatestImage();
|
|
|
LogLoadPlan logPlan = createLogLoadPlan(recoveryImage.txId, Long.MAX_VALUE);
|
|
|
|
|
|
return new TransactionalLoadPlan(recoveryImage,
|
|
@@ -233,7 +195,7 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
LogLoadPlan createLogLoadPlan(long sinceTxId, long maxStartTxId) throws IOException {
|
|
|
long expectedTxId = sinceTxId + 1;
|
|
|
|
|
|
- List<FoundEditLog> recoveryLogs = new ArrayList<FoundEditLog>();
|
|
|
+ List<EditLogFile> recoveryLogs = new ArrayList<EditLogFile>();
|
|
|
|
|
|
SortedMap<Long, LogGroup> tailGroups = logGroups.tailMap(expectedTxId);
|
|
|
if (logGroups.size() > tailGroups.size()) {
|
|
@@ -312,10 +274,10 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
for (LogGroup g : logGroups.values()) {
|
|
|
if (!g.hasFinalized) continue;
|
|
|
|
|
|
- FoundEditLog fel = g.getBestNonCorruptLog();
|
|
|
+ EditLogFile fel = g.getBestNonCorruptLog();
|
|
|
if (fel.getLastTxId() < sinceTxId) continue;
|
|
|
|
|
|
- logs.add(new RemoteEditLog(fel.getStartTxId(),
|
|
|
+ logs.add(new RemoteEditLog(fel.getFirstTxId(),
|
|
|
fel.getLastTxId()));
|
|
|
}
|
|
|
|
|
@@ -330,7 +292,7 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
*/
|
|
|
static class LogGroup {
|
|
|
long startTxId;
|
|
|
- List<FoundEditLog> logs = new ArrayList<FoundEditLog>();;
|
|
|
+ List<EditLogFile> logs = new ArrayList<EditLogFile>();;
|
|
|
private Set<Long> endTxIds = new TreeSet<Long>();
|
|
|
private boolean hasInProgress = false;
|
|
|
private boolean hasFinalized = false;
|
|
@@ -339,15 +301,15 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
this.startTxId = startTxId;
|
|
|
}
|
|
|
|
|
|
- FoundEditLog getBestNonCorruptLog() {
|
|
|
+ EditLogFile getBestNonCorruptLog() {
|
|
|
// First look for non-corrupt finalized logs
|
|
|
- for (FoundEditLog log : logs) {
|
|
|
+ for (EditLogFile log : logs) {
|
|
|
if (!log.isCorrupt() && !log.isInProgress()) {
|
|
|
return log;
|
|
|
}
|
|
|
}
|
|
|
// Then look for non-corrupt in-progress logs
|
|
|
- for (FoundEditLog log : logs) {
|
|
|
+ for (EditLogFile log : logs) {
|
|
|
if (!log.isCorrupt()) {
|
|
|
return log;
|
|
|
}
|
|
@@ -364,7 +326,7 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
* @return true if we can determine the last txid in this log group.
|
|
|
*/
|
|
|
boolean hasKnownLastTxId() {
|
|
|
- for (FoundEditLog log : logs) {
|
|
|
+ for (EditLogFile log : logs) {
|
|
|
if (!log.isInProgress()) {
|
|
|
return true;
|
|
|
}
|
|
@@ -378,24 +340,24 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
* {@see #hasKnownLastTxId()}
|
|
|
*/
|
|
|
long getLastTxId() {
|
|
|
- for (FoundEditLog log : logs) {
|
|
|
+ for (EditLogFile log : logs) {
|
|
|
if (!log.isInProgress()) {
|
|
|
- return log.lastTxId;
|
|
|
+ return log.getLastTxId();
|
|
|
}
|
|
|
}
|
|
|
throw new IllegalStateException("LogGroup only has in-progress logs");
|
|
|
}
|
|
|
|
|
|
|
|
|
- void add(FoundEditLog log) {
|
|
|
- assert log.getStartTxId() == startTxId;
|
|
|
+ void add(EditLogFile log) {
|
|
|
+ assert log.getFirstTxId() == startTxId;
|
|
|
logs.add(log);
|
|
|
|
|
|
if (log.isInProgress()) {
|
|
|
hasInProgress = true;
|
|
|
} else {
|
|
|
hasFinalized = true;
|
|
|
- endTxIds.add(log.lastTxId);
|
|
|
+ endTxIds.add(log.getLastTxId());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -422,7 +384,7 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
* The in-progress logs in this case should be considered corrupt.
|
|
|
*/
|
|
|
private void planMixedLogRecovery() throws IOException {
|
|
|
- for (FoundEditLog log : logs) {
|
|
|
+ for (EditLogFile log : logs) {
|
|
|
if (log.isInProgress()) {
|
|
|
LOG.warn("Log at " + log.getFile() + " is in progress, but " +
|
|
|
"other logs starting at the same txid " + startTxId +
|
|
@@ -446,7 +408,7 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
"crash)");
|
|
|
if (logs.size() == 1) {
|
|
|
// Only one log, it's our only choice!
|
|
|
- FoundEditLog log = logs.get(0);
|
|
|
+ EditLogFile log = logs.get(0);
|
|
|
if (log.validateLog().numTransactions == 0) {
|
|
|
// If it has no transactions, we should consider it corrupt just
|
|
|
// to be conservative.
|
|
@@ -459,7 +421,7 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
}
|
|
|
|
|
|
long maxValidTxnCount = Long.MIN_VALUE;
|
|
|
- for (FoundEditLog log : logs) {
|
|
|
+ for (EditLogFile log : logs) {
|
|
|
long validTxnCount = log.validateLog().numTransactions;
|
|
|
LOG.warn(" Log " + log.getFile() +
|
|
|
" valid txns=" + validTxnCount +
|
|
@@ -467,7 +429,7 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
maxValidTxnCount = Math.max(maxValidTxnCount, validTxnCount);
|
|
|
}
|
|
|
|
|
|
- for (FoundEditLog log : logs) {
|
|
|
+ for (EditLogFile log : logs) {
|
|
|
long txns = log.validateLog().numTransactions;
|
|
|
if (txns < maxValidTxnCount) {
|
|
|
LOG.warn("Marking log at " + log.getFile() + " as corrupt since " +
|
|
@@ -499,7 +461,7 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
}
|
|
|
|
|
|
void recover() throws IOException {
|
|
|
- for (FoundEditLog log : logs) {
|
|
|
+ for (EditLogFile log : logs) {
|
|
|
if (log.isCorrupt()) {
|
|
|
log.moveAsideCorruptFile();
|
|
|
} else if (log.isInProgress()) {
|
|
@@ -508,131 +470,12 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * Record of an image that has been located and had its filename parsed.
|
|
|
- */
|
|
|
- static class FoundFSImage {
|
|
|
- final StorageDirectory sd;
|
|
|
- final long txId;
|
|
|
- private final File file;
|
|
|
-
|
|
|
- FoundFSImage(StorageDirectory sd, File file, long txId) {
|
|
|
- assert txId >= 0 : "Invalid txid on " + file +": " + txId;
|
|
|
-
|
|
|
- this.sd = sd;
|
|
|
- this.txId = txId;
|
|
|
- this.file = file;
|
|
|
- }
|
|
|
-
|
|
|
- File getFile() {
|
|
|
- return file;
|
|
|
- }
|
|
|
-
|
|
|
- public long getTxId() {
|
|
|
- return txId;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public String toString() {
|
|
|
- return file.toString();
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
- /**
|
|
|
- * Record of an edit log that has been located and had its filename parsed.
|
|
|
- */
|
|
|
- static class FoundEditLog {
|
|
|
- File file;
|
|
|
- final long startTxId;
|
|
|
- long lastTxId;
|
|
|
-
|
|
|
- private EditLogValidation cachedValidation = null;
|
|
|
- private boolean isCorrupt = false;
|
|
|
-
|
|
|
- static final long UNKNOWN_END = -1;
|
|
|
-
|
|
|
- FoundEditLog(File file,
|
|
|
- long startTxId, long endTxId) {
|
|
|
- assert endTxId == UNKNOWN_END || endTxId >= startTxId;
|
|
|
- assert startTxId > 0;
|
|
|
- assert file != null;
|
|
|
-
|
|
|
- this.startTxId = startTxId;
|
|
|
- this.lastTxId = endTxId;
|
|
|
- this.file = file;
|
|
|
- }
|
|
|
-
|
|
|
- public void finalizeLog() throws IOException {
|
|
|
- long numTransactions = validateLog().numTransactions;
|
|
|
- long lastTxId = startTxId + numTransactions - 1;
|
|
|
- File dst = new File(file.getParentFile(),
|
|
|
- NNStorage.getFinalizedEditsFileName(startTxId, lastTxId));
|
|
|
- LOG.info("Finalizing edits log " + file + " by renaming to "
|
|
|
- + dst.getName());
|
|
|
- if (!file.renameTo(dst)) {
|
|
|
- throw new IOException("Couldn't finalize log " +
|
|
|
- file + " to " + dst);
|
|
|
- }
|
|
|
- this.lastTxId = lastTxId;
|
|
|
- file = dst;
|
|
|
- }
|
|
|
-
|
|
|
- long getStartTxId() {
|
|
|
- return startTxId;
|
|
|
- }
|
|
|
-
|
|
|
- long getLastTxId() {
|
|
|
- return lastTxId;
|
|
|
- }
|
|
|
-
|
|
|
- EditLogValidation validateLog() throws IOException {
|
|
|
- if (cachedValidation == null) {
|
|
|
- cachedValidation = EditLogFileInputStream.validateEditLog(file);
|
|
|
- }
|
|
|
- return cachedValidation;
|
|
|
- }
|
|
|
-
|
|
|
- boolean isInProgress() {
|
|
|
- return (lastTxId == UNKNOWN_END);
|
|
|
- }
|
|
|
-
|
|
|
- File getFile() {
|
|
|
- return file;
|
|
|
- }
|
|
|
-
|
|
|
- void markCorrupt() {
|
|
|
- isCorrupt = true;
|
|
|
- }
|
|
|
-
|
|
|
- boolean isCorrupt() {
|
|
|
- return isCorrupt;
|
|
|
- }
|
|
|
-
|
|
|
- void moveAsideCorruptFile() throws IOException {
|
|
|
- assert isCorrupt;
|
|
|
-
|
|
|
- File src = file;
|
|
|
- File dst = new File(src.getParent(), src.getName() + ".corrupt");
|
|
|
- boolean success = src.renameTo(dst);
|
|
|
- if (!success) {
|
|
|
- throw new IOException(
|
|
|
- "Couldn't rename corrupt log " + src + " to " + dst);
|
|
|
- }
|
|
|
- file = dst;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public String toString() {
|
|
|
- return file.toString();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
static class TransactionalLoadPlan extends LoadPlan {
|
|
|
- final FoundFSImage image;
|
|
|
+ final FSImageFile image;
|
|
|
final LogLoadPlan logPlan;
|
|
|
|
|
|
- public TransactionalLoadPlan(FoundFSImage image,
|
|
|
+ public TransactionalLoadPlan(FSImageFile image,
|
|
|
LogLoadPlan logPlan) {
|
|
|
super();
|
|
|
this.image = image;
|
|
@@ -662,10 +505,10 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
}
|
|
|
|
|
|
static class LogLoadPlan {
|
|
|
- final List<FoundEditLog> editLogs;
|
|
|
+ final List<EditLogFile> editLogs;
|
|
|
final List<LogGroup> logGroupsToRecover;
|
|
|
|
|
|
- LogLoadPlan(List<FoundEditLog> editLogs,
|
|
|
+ LogLoadPlan(List<EditLogFile> editLogs,
|
|
|
List<LogGroup> logGroupsToRecover) {
|
|
|
this.editLogs = editLogs;
|
|
|
this.logGroupsToRecover = logGroupsToRecover;
|
|
@@ -679,7 +522,7 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
|
|
|
|
|
|
public List<File> getEditsFiles() {
|
|
|
List<File> ret = new ArrayList<File>();
|
|
|
- for (FoundEditLog log : editLogs) {
|
|
|
+ for (EditLogFile log : editLogs) {
|
|
|
ret.add(log.getFile());
|
|
|
}
|
|
|
return ret;
|