|
@@ -17,12 +17,12 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
+import static org.apache.hadoop.hdfs.server.common.Util.now;
|
|
|
+
|
|
|
import java.io.IOException;
|
|
|
+import java.util.ArrayList;
|
|
|
import java.util.Collection;
|
|
|
-import java.util.Collections;
|
|
|
-import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
-import java.util.SortedSet;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -34,25 +34,17 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
|
|
|
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
|
|
-import static org.apache.hadoop.hdfs.server.common.Util.now;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.JournalManager.CorruptionException;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
|
|
|
-import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
|
|
|
-import org.apache.hadoop.security.token.delegation.DelegationKey;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
+import org.apache.hadoop.security.token.delegation.DelegationKey;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
-import com.google.common.collect.ImmutableList;
|
|
|
-import com.google.common.collect.ImmutableListMultimap;
|
|
|
-import com.google.common.collect.Lists;
|
|
|
-import com.google.common.collect.Multimaps;
|
|
|
-import com.google.common.collect.Sets;
|
|
|
-
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
|
|
|
|
|
|
/**
|
|
|
* FSEditLog maintains a log of the namespace modifications.
|
|
@@ -62,9 +54,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
|
|
|
@InterfaceStability.Evolving
|
|
|
public class FSEditLog {
|
|
|
|
|
|
- static final String NO_JOURNAL_STREAMS_WARNING = "!!! WARNING !!!" +
|
|
|
- " File system changes are not persistent. No journal streams.";
|
|
|
-
|
|
|
static final Log LOG = LogFactory.getLog(FSEditLog.class);
|
|
|
|
|
|
/**
|
|
@@ -82,10 +71,11 @@ public class FSEditLog {
|
|
|
CLOSED;
|
|
|
}
|
|
|
private State state = State.UNINITIALIZED;
|
|
|
+
|
|
|
+ //initialize
|
|
|
+ final private JournalSet journalSet;
|
|
|
+ private EditLogOutputStream editLogStream = null;
|
|
|
|
|
|
-
|
|
|
- private List<JournalAndStream> journals = Lists.newArrayList();
|
|
|
-
|
|
|
// a monotonically increasing counter that represents transactionIds.
|
|
|
private long txid = 0;
|
|
|
|
|
@@ -137,15 +127,15 @@ public class FSEditLog {
|
|
|
this.storage = storage;
|
|
|
metrics = NameNode.getNameNodeMetrics();
|
|
|
lastPrintTime = now();
|
|
|
-
|
|
|
+
|
|
|
+ this.journalSet = new JournalSet();
|
|
|
for (StorageDirectory sd : storage.dirIterable(NameNodeDirType.EDITS)) {
|
|
|
- journals.add(new JournalAndStream(new FileJournalManager(sd)));
|
|
|
+ journalSet.add(new FileJournalManager(sd));
|
|
|
}
|
|
|
|
|
|
- if (journals.isEmpty()) {
|
|
|
+ if (journalSet.isEmpty()) {
|
|
|
LOG.error("No edits directories configured!");
|
|
|
- }
|
|
|
-
|
|
|
+ }
|
|
|
state = State.BETWEEN_LOG_SEGMENTS;
|
|
|
}
|
|
|
|
|
@@ -172,9 +162,8 @@ public class FSEditLog {
|
|
|
LOG.warn("Closing log when already closed", new Exception());
|
|
|
return;
|
|
|
}
|
|
|
-
|
|
|
if (state == State.IN_SEGMENT) {
|
|
|
- assert !journals.isEmpty();
|
|
|
+ assert editLogStream != null;
|
|
|
waitForSyncToFinish();
|
|
|
endCurrentLogSegment(true);
|
|
|
}
|
|
@@ -193,20 +182,14 @@ public class FSEditLog {
|
|
|
// wait if an automatic sync is scheduled
|
|
|
waitIfAutoSyncScheduled();
|
|
|
|
|
|
- if (journals.isEmpty()) {
|
|
|
- throw new java.lang.IllegalStateException(NO_JOURNAL_STREAMS_WARNING);
|
|
|
- }
|
|
|
-
|
|
|
long start = beginTransaction();
|
|
|
op.setTransactionId(txid);
|
|
|
|
|
|
- mapJournalsAndReportErrors(new JournalClosure() {
|
|
|
- @Override
|
|
|
- public void apply(JournalAndStream jas) throws IOException {
|
|
|
- if (!jas.isActive()) return;
|
|
|
- jas.stream.write(op);
|
|
|
- }
|
|
|
- }, "logging edit");
|
|
|
+ try {
|
|
|
+ editLogStream.write(op);
|
|
|
+ } catch (IOException ex) {
|
|
|
+ // All journals failed, it is handled in logSync.
|
|
|
+ }
|
|
|
|
|
|
endTransaction(start);
|
|
|
|
|
@@ -251,14 +234,7 @@ public class FSEditLog {
|
|
|
* @return true if any of the edit stream says that it should sync
|
|
|
*/
|
|
|
private boolean shouldForceSync() {
|
|
|
- for (JournalAndStream jas : journals) {
|
|
|
- if (!jas.isActive()) continue;
|
|
|
-
|
|
|
- if (jas.getCurrentStream().shouldForceSync()) {
|
|
|
- return true;
|
|
|
- }
|
|
|
- }
|
|
|
- return false;
|
|
|
+ return editLogStream.shouldForceSync();
|
|
|
}
|
|
|
|
|
|
private long beginTransaction() {
|
|
@@ -322,7 +298,7 @@ public class FSEditLog {
|
|
|
* NOTE: this should be done while holding the FSNamesystem lock, or
|
|
|
* else more operations can start writing while this is in progress.
|
|
|
*/
|
|
|
- void logSyncAll() throws IOException {
|
|
|
+ void logSyncAll() {
|
|
|
// Record the most recent transaction ID as our own id
|
|
|
synchronized (this) {
|
|
|
TransactionId id = myTransactionId.get();
|
|
@@ -366,74 +342,73 @@ public class FSEditLog {
|
|
|
// Fetch the transactionId of this thread.
|
|
|
long mytxid = myTransactionId.get().txid;
|
|
|
|
|
|
- List<JournalAndStream> candidateJournals =
|
|
|
- Lists.newArrayListWithCapacity(journals.size());
|
|
|
- List<JournalAndStream> badJournals = Lists.newArrayList();
|
|
|
-
|
|
|
boolean sync = false;
|
|
|
try {
|
|
|
+ EditLogOutputStream logStream = null;
|
|
|
synchronized (this) {
|
|
|
try {
|
|
|
- printStatistics(false);
|
|
|
-
|
|
|
- // if somebody is already syncing, then wait
|
|
|
- while (mytxid > synctxid && isSyncRunning) {
|
|
|
- try {
|
|
|
- wait(1000);
|
|
|
- } catch (InterruptedException ie) {
|
|
|
+ printStatistics(false);
|
|
|
+
|
|
|
+ // if somebody is already syncing, then wait
|
|
|
+ while (mytxid > synctxid && isSyncRunning) {
|
|
|
+ try {
|
|
|
+ wait(1000);
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ }
|
|
|
}
|
|
|
- }
|
|
|
|
|
|
- //
|
|
|
- // If this transaction was already flushed, then nothing to do
|
|
|
- //
|
|
|
- if (mytxid <= synctxid) {
|
|
|
- numTransactionsBatchedInSync++;
|
|
|
- if (metrics != null) // Metrics is non-null only when used inside name node
|
|
|
- metrics.incrTransactionsBatchedInSync();
|
|
|
- return;
|
|
|
- }
|
|
|
+ //
|
|
|
+ // If this transaction was already flushed, then nothing to do
|
|
|
+ //
|
|
|
+ if (mytxid <= synctxid) {
|
|
|
+ numTransactionsBatchedInSync++;
|
|
|
+ if (metrics != null) {
|
|
|
+ // Metrics is non-null only when used inside name node
|
|
|
+ metrics.incrTransactionsBatchedInSync();
|
|
|
+ }
|
|
|
+ return;
|
|
|
+ }
|
|
|
|
|
|
- // now, this thread will do the sync
|
|
|
- syncStart = txid;
|
|
|
- isSyncRunning = true;
|
|
|
- sync = true;
|
|
|
+ // now, this thread will do the sync
|
|
|
+ syncStart = txid;
|
|
|
+ isSyncRunning = true;
|
|
|
+ sync = true;
|
|
|
|
|
|
- // swap buffers
|
|
|
- assert !journals.isEmpty() : "no editlog streams";
|
|
|
-
|
|
|
- for (JournalAndStream jas : journals) {
|
|
|
- if (!jas.isActive()) continue;
|
|
|
+ // swap buffers
|
|
|
try {
|
|
|
- jas.getCurrentStream().setReadyToFlush();
|
|
|
- candidateJournals.add(jas);
|
|
|
- } catch (IOException ie) {
|
|
|
- LOG.error("Unable to get ready to flush.", ie);
|
|
|
- badJournals.add(jas);
|
|
|
+ if (journalSet.isEmpty()) {
|
|
|
+ throw new IOException("No journals available to flush");
|
|
|
+ }
|
|
|
+ editLogStream.setReadyToFlush();
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.fatal("Could not sync any journal to persistent storage. "
|
|
|
+ + "Unsynced transactions: " + (txid - synctxid),
|
|
|
+ new Exception());
|
|
|
+ runtime.exit(1);
|
|
|
}
|
|
|
- }
|
|
|
} finally {
|
|
|
// Prevent RuntimeException from blocking other log edit write
|
|
|
doneWithAutoSyncScheduling();
|
|
|
}
|
|
|
+ //editLogStream may become null,
|
|
|
+ //so store a local variable for flush.
|
|
|
+ logStream = editLogStream;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
// do the sync
|
|
|
long start = now();
|
|
|
- for (JournalAndStream jas : candidateJournals) {
|
|
|
- if (!jas.isActive()) continue;
|
|
|
- try {
|
|
|
- jas.getCurrentStream().flush();
|
|
|
- } catch (IOException ie) {
|
|
|
- LOG.error("Unable to sync edit log.", ie);
|
|
|
- //
|
|
|
- // remember the streams that encountered an error.
|
|
|
- //
|
|
|
- badJournals.add(jas);
|
|
|
+ try {
|
|
|
+ if (logStream != null) {
|
|
|
+ logStream.flush();
|
|
|
+ }
|
|
|
+ } catch (IOException ex) {
|
|
|
+ synchronized (this) {
|
|
|
+ LOG.fatal("Could not sync any journal to persistent storage. "
|
|
|
+ + "Unsynced transactions: " + (txid - synctxid), new Exception());
|
|
|
+ runtime.exit(1);
|
|
|
}
|
|
|
}
|
|
|
long elapsed = now() - start;
|
|
|
- disableAndReportErrorOnJournals(badJournals);
|
|
|
|
|
|
if (metrics != null) { // Metrics non-null only when used inside name node
|
|
|
metrics.addSync(elapsed);
|
|
@@ -443,13 +418,6 @@ public class FSEditLog {
|
|
|
// Prevent RuntimeException from blocking other log edit sync
|
|
|
synchronized (this) {
|
|
|
if (sync) {
|
|
|
- if (badJournals.size() >= journals.size()) {
|
|
|
- LOG.fatal("Could not sync any journal to persistent storage. " +
|
|
|
- "Unsynced transactions: " + (txid - synctxid),
|
|
|
- new Exception());
|
|
|
- runtime.exit(1);
|
|
|
- }
|
|
|
-
|
|
|
synctxid = syncStart;
|
|
|
isSyncRunning = false;
|
|
|
}
|
|
@@ -466,9 +434,6 @@ public class FSEditLog {
|
|
|
if (lastPrintTime + 60000 > now && !force) {
|
|
|
return;
|
|
|
}
|
|
|
- if (journals.isEmpty()) {
|
|
|
- return;
|
|
|
- }
|
|
|
lastPrintTime = now;
|
|
|
StringBuilder buf = new StringBuilder();
|
|
|
buf.append("Number of transactions: ");
|
|
@@ -478,20 +443,9 @@ public class FSEditLog {
|
|
|
buf.append("Number of transactions batched in Syncs: ");
|
|
|
buf.append(numTransactionsBatchedInSync);
|
|
|
buf.append(" Number of syncs: ");
|
|
|
- for (JournalAndStream jas : journals) {
|
|
|
- if (!jas.isActive()) continue;
|
|
|
- buf.append(jas.getCurrentStream().getNumSync());
|
|
|
- break;
|
|
|
- }
|
|
|
-
|
|
|
+ buf.append(editLogStream.getNumSync());
|
|
|
buf.append(" SyncTimes(ms): ");
|
|
|
-
|
|
|
- for (JournalAndStream jas : journals) {
|
|
|
- if (!jas.isActive()) continue;
|
|
|
- EditLogOutputStream eStream = jas.getCurrentStream();
|
|
|
- buf.append(eStream.getTotalSyncTime());
|
|
|
- buf.append(" ");
|
|
|
- }
|
|
|
+ buf.append(journalSet.getSyncTimes());
|
|
|
LOG.info(buf);
|
|
|
}
|
|
|
|
|
@@ -664,7 +618,6 @@ public class FSEditLog {
|
|
|
* log delegation token to edit log
|
|
|
* @param id DelegationTokenIdentifier
|
|
|
* @param expiryTime of the token
|
|
|
- * @return
|
|
|
*/
|
|
|
void logGetDelegationToken(DelegationTokenIdentifier id,
|
|
|
long expiryTime) {
|
|
@@ -702,25 +655,12 @@ public class FSEditLog {
|
|
|
logEdit(op);
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * @return the number of active (non-failed) journals
|
|
|
- */
|
|
|
- private int countActiveJournals() {
|
|
|
- int count = 0;
|
|
|
- for (JournalAndStream jas : journals) {
|
|
|
- if (jas.isActive()) {
|
|
|
- count++;
|
|
|
- }
|
|
|
- }
|
|
|
- return count;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Used only by unit tests.
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
List<JournalAndStream> getJournals() {
|
|
|
- return journals;
|
|
|
+ return journalSet.getAllJournalStreams();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -734,62 +674,9 @@ public class FSEditLog {
|
|
|
/**
|
|
|
* Return a manifest of what finalized edit logs are available
|
|
|
*/
|
|
|
- public synchronized RemoteEditLogManifest getEditLogManifest(
|
|
|
- long fromTxId) throws IOException {
|
|
|
- // Collect RemoteEditLogs available from each FileJournalManager
|
|
|
- List<RemoteEditLog> allLogs = Lists.newArrayList();
|
|
|
- for (JournalAndStream j : journals) {
|
|
|
- if (j.getManager() instanceof FileJournalManager) {
|
|
|
- FileJournalManager fjm = (FileJournalManager)j.getManager();
|
|
|
- try {
|
|
|
- allLogs.addAll(fjm.getRemoteEditLogs(fromTxId));
|
|
|
- } catch (Throwable t) {
|
|
|
- LOG.warn("Cannot list edit logs in " + fjm, t);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // Group logs by their starting txid
|
|
|
- ImmutableListMultimap<Long, RemoteEditLog> logsByStartTxId =
|
|
|
- Multimaps.index(allLogs, RemoteEditLog.GET_START_TXID);
|
|
|
- long curStartTxId = fromTxId;
|
|
|
-
|
|
|
- List<RemoteEditLog> logs = Lists.newArrayList();
|
|
|
- while (true) {
|
|
|
- ImmutableList<RemoteEditLog> logGroup = logsByStartTxId.get(curStartTxId);
|
|
|
- if (logGroup.isEmpty()) {
|
|
|
- // we have a gap in logs - for example because we recovered some old
|
|
|
- // storage directory with ancient logs. Clear out any logs we've
|
|
|
- // accumulated so far, and then skip to the next segment of logs
|
|
|
- // after the gap.
|
|
|
- SortedSet<Long> startTxIds = Sets.newTreeSet(logsByStartTxId.keySet());
|
|
|
- startTxIds = startTxIds.tailSet(curStartTxId);
|
|
|
- if (startTxIds.isEmpty()) {
|
|
|
- break;
|
|
|
- } else {
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Found gap in logs at " + curStartTxId + ": " +
|
|
|
- "not returning previous logs in manifest.");
|
|
|
- }
|
|
|
- logs.clear();
|
|
|
- curStartTxId = startTxIds.first();
|
|
|
- continue;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // Find the one that extends the farthest forward
|
|
|
- RemoteEditLog bestLog = Collections.max(logGroup);
|
|
|
- logs.add(bestLog);
|
|
|
- // And then start looking from after that point
|
|
|
- curStartTxId = bestLog.getEndTxId() + 1;
|
|
|
- }
|
|
|
- RemoteEditLogManifest ret = new RemoteEditLogManifest(logs);
|
|
|
-
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Generated manifest for logs since " + fromTxId + ":"
|
|
|
- + ret);
|
|
|
- }
|
|
|
- return ret;
|
|
|
+ public synchronized RemoteEditLogManifest getEditLogManifest(long fromTxId)
|
|
|
+ throws IOException {
|
|
|
+ return journalSet.getEditLogManifest(fromTxId);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -832,14 +719,9 @@ public class FSEditLog {
|
|
|
// See HDFS-2174.
|
|
|
storage.attemptRestoreRemovedStorage();
|
|
|
|
|
|
- mapJournalsAndReportErrors(new JournalClosure() {
|
|
|
- @Override
|
|
|
- public void apply(JournalAndStream jas) throws IOException {
|
|
|
- jas.startLogSegment(segmentTxId);
|
|
|
- }
|
|
|
- }, "starting log segment " + segmentTxId);
|
|
|
-
|
|
|
- if (countActiveJournals() == 0) {
|
|
|
+ try {
|
|
|
+ editLogStream = journalSet.startLogSegment(segmentTxId);
|
|
|
+ } catch (IOException ex) {
|
|
|
throw new IOException("Unable to start log segment " +
|
|
|
segmentTxId + ": no journals successfully started.");
|
|
|
}
|
|
@@ -873,14 +755,12 @@ public class FSEditLog {
|
|
|
|
|
|
final long lastTxId = getLastWrittenTxId();
|
|
|
|
|
|
- mapJournalsAndReportErrors(new JournalClosure() {
|
|
|
- @Override
|
|
|
- public void apply(JournalAndStream jas) throws IOException {
|
|
|
- if (jas.isActive()) {
|
|
|
- jas.close(lastTxId);
|
|
|
- }
|
|
|
- }
|
|
|
- }, "ending log segment");
|
|
|
+ try {
|
|
|
+ journalSet.finalizeLogSegment(curSegmentTxId, lastTxId);
|
|
|
+ editLogStream = null;
|
|
|
+ } catch (IOException e) {
|
|
|
+ //All journals have failed, it will be handled in logSync.
|
|
|
+ }
|
|
|
|
|
|
state = State.BETWEEN_LOG_SEGMENTS;
|
|
|
}
|
|
@@ -889,14 +769,15 @@ public class FSEditLog {
|
|
|
* Abort all current logs. Called from the backup node.
|
|
|
*/
|
|
|
synchronized void abortCurrentLogSegment() {
|
|
|
- mapJournalsAndReportErrors(new JournalClosure() {
|
|
|
-
|
|
|
- @Override
|
|
|
- public void apply(JournalAndStream jas) throws IOException {
|
|
|
- jas.abort();
|
|
|
+ try {
|
|
|
+ //Check for null, as abort can be called any time.
|
|
|
+ if (editLogStream != null) {
|
|
|
+ editLogStream.abort();
|
|
|
+ editLogStream = null;
|
|
|
}
|
|
|
- }, "aborting all streams");
|
|
|
- state = State.BETWEEN_LOG_SEGMENTS;
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("All journals failed to abort", e);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -912,13 +793,12 @@ public class FSEditLog {
|
|
|
"cannot purge logs older than txid " + minTxIdToKeep +
|
|
|
" when current segment starts at " + curSegmentTxId;
|
|
|
}
|
|
|
-
|
|
|
- mapJournalsAndReportErrors(new JournalClosure() {
|
|
|
- @Override
|
|
|
- public void apply(JournalAndStream jas) throws IOException {
|
|
|
- jas.manager.purgeLogsOlderThan(minTxIdToKeep);
|
|
|
- }
|
|
|
- }, "purging logs older than " + minTxIdToKeep);
|
|
|
+
|
|
|
+ try {
|
|
|
+ journalSet.purgeLogsOlderThan(minTxIdToKeep);
|
|
|
+ } catch (IOException ex) {
|
|
|
+ //All journals have failed, it will be handled in logSync.
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
|
|
@@ -946,9 +826,7 @@ public class FSEditLog {
|
|
|
|
|
|
// sets the initial capacity of the flush buffer.
|
|
|
public void setOutputBufferCapacity(int size) {
|
|
|
- for (JournalAndStream jas : journals) {
|
|
|
- jas.manager.setOutputBufferCapacity(size);
|
|
|
- }
|
|
|
+ journalSet.setOutputBufferCapacity(size);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -969,7 +847,7 @@ public class FSEditLog {
|
|
|
if(bnReg.isRole(NamenodeRole.CHECKPOINT))
|
|
|
return; // checkpoint node does not stream edits
|
|
|
|
|
|
- JournalAndStream jas = findBackupJournalAndStream(bnReg);
|
|
|
+ JournalManager jas = findBackupJournal(bnReg);
|
|
|
if (jas != null) {
|
|
|
// already registered
|
|
|
LOG.info("Backup node " + bnReg + " re-registers");
|
|
@@ -978,35 +856,29 @@ public class FSEditLog {
|
|
|
|
|
|
LOG.info("Registering new backup node: " + bnReg);
|
|
|
BackupJournalManager bjm = new BackupJournalManager(bnReg, nnReg);
|
|
|
- journals.add(new JournalAndStream(bjm));
|
|
|
+ journalSet.add(bjm);
|
|
|
}
|
|
|
|
|
|
- synchronized void releaseBackupStream(NamenodeRegistration registration) {
|
|
|
- for (Iterator<JournalAndStream> iter = journals.iterator();
|
|
|
- iter.hasNext();) {
|
|
|
- JournalAndStream jas = iter.next();
|
|
|
- if (jas.manager instanceof BackupJournalManager &&
|
|
|
- ((BackupJournalManager)jas.manager).matchesRegistration(
|
|
|
- registration)) {
|
|
|
- jas.abort();
|
|
|
- LOG.info("Removing backup journal " + jas);
|
|
|
- iter.remove();
|
|
|
- }
|
|
|
+ synchronized void releaseBackupStream(NamenodeRegistration registration)
|
|
|
+ throws IOException {
|
|
|
+ BackupJournalManager bjm = this.findBackupJournal(registration);
|
|
|
+ if (bjm != null) {
|
|
|
+ LOG.info("Removing backup journal " + bjm);
|
|
|
+ journalSet.remove(bjm);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Find the JournalAndStream associated with this BackupNode.
|
|
|
+ *
|
|
|
* @return null if it cannot be found
|
|
|
*/
|
|
|
- private synchronized JournalAndStream findBackupJournalAndStream(
|
|
|
+ private synchronized BackupJournalManager findBackupJournal(
|
|
|
NamenodeRegistration bnReg) {
|
|
|
- for (JournalAndStream jas : journals) {
|
|
|
- if (jas.manager instanceof BackupJournalManager) {
|
|
|
- BackupJournalManager bjm = (BackupJournalManager)jas.manager;
|
|
|
- if (bjm.matchesRegistration(bnReg)) {
|
|
|
- return jas;
|
|
|
- }
|
|
|
+ for (JournalManager bjm : journalSet.getJournalManagers()) {
|
|
|
+ if ((bjm instanceof BackupJournalManager)
|
|
|
+ && ((BackupJournalManager) bjm).matchesRegistration(bnReg)) {
|
|
|
+ return (BackupJournalManager) bjm;
|
|
|
}
|
|
|
}
|
|
|
return null;
|
|
@@ -1018,124 +890,24 @@ public class FSEditLog {
|
|
|
*/
|
|
|
synchronized void logEdit(final int length, final byte[] data) {
|
|
|
long start = beginTransaction();
|
|
|
-
|
|
|
- mapJournalsAndReportErrors(new JournalClosure() {
|
|
|
- @Override
|
|
|
- public void apply(JournalAndStream jas) throws IOException {
|
|
|
- if (jas.isActive()) {
|
|
|
- jas.getCurrentStream().writeRaw(data, 0, length); // TODO writeRaw
|
|
|
- }
|
|
|
- }
|
|
|
- }, "Logging edit");
|
|
|
-
|
|
|
- endTransaction(start);
|
|
|
- }
|
|
|
|
|
|
- //// Iteration across journals
|
|
|
- private interface JournalClosure {
|
|
|
- public void apply(JournalAndStream jas) throws IOException;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Apply the given function across all of the journal managers, disabling
|
|
|
- * any for which the closure throws an IOException.
|
|
|
- * @param status message used for logging errors (e.g. "opening journal")
|
|
|
- */
|
|
|
- private void mapJournalsAndReportErrors(
|
|
|
- JournalClosure closure, String status) {
|
|
|
- List<JournalAndStream> badJAS = Lists.newLinkedList();
|
|
|
- for (JournalAndStream jas : journals) {
|
|
|
- try {
|
|
|
- closure.apply(jas);
|
|
|
- } catch (Throwable t) {
|
|
|
- LOG.error("Error " + status + " (journal " + jas + ")", t);
|
|
|
- badJAS.add(jas);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- disableAndReportErrorOnJournals(badJAS);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Called when some journals experience an error in some operation.
|
|
|
- * This propagates errors to the storage level.
|
|
|
- */
|
|
|
- private void disableAndReportErrorOnJournals(List<JournalAndStream> badJournals) {
|
|
|
- if (badJournals == null || badJournals.isEmpty()) {
|
|
|
- return; // nothing to do
|
|
|
- }
|
|
|
-
|
|
|
- for (JournalAndStream j : badJournals) {
|
|
|
- LOG.error("Disabling journal " + j);
|
|
|
- j.abort();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Find the best editlog input stream to read from txid. In this case
|
|
|
- * best means the editlog which has the largest continuous range of
|
|
|
- * transactions starting from the transaction id, fromTxId.
|
|
|
- *
|
|
|
- * If a journal throws an CorruptionException while reading from a txn id,
|
|
|
- * it means that it has more transactions, but can't find any from fromTxId.
|
|
|
- * If this is the case and no other journal has transactions, we should throw
|
|
|
- * an exception as it means more transactions exist, we just can't load them.
|
|
|
- *
|
|
|
- * @param fromTxId Transaction id to start from.
|
|
|
- * @return a edit log input stream with tranactions fromTxId
|
|
|
- * or null if no more exist
|
|
|
- */
|
|
|
- private EditLogInputStream selectStream(long fromTxId)
|
|
|
- throws IOException {
|
|
|
- JournalManager bestjm = null;
|
|
|
- long bestjmNumTxns = 0;
|
|
|
- CorruptionException corruption = null;
|
|
|
-
|
|
|
- for (JournalAndStream jas : journals) {
|
|
|
- JournalManager candidate = jas.getManager();
|
|
|
- long candidateNumTxns = 0;
|
|
|
- try {
|
|
|
- candidateNumTxns = candidate.getNumberOfTransactions(fromTxId);
|
|
|
- } catch (CorruptionException ce) {
|
|
|
- corruption = ce;
|
|
|
- } catch (IOException ioe) {
|
|
|
- LOG.warn("Error reading number of transactions from " + candidate);
|
|
|
- continue; // error reading disk, just skip
|
|
|
- }
|
|
|
-
|
|
|
- if (candidateNumTxns > bestjmNumTxns) {
|
|
|
- bestjm = candidate;
|
|
|
- bestjmNumTxns = candidateNumTxns;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
- if (bestjm == null) {
|
|
|
- /**
|
|
|
- * If all candidates either threw a CorruptionException or
|
|
|
- * found 0 transactions, then a gap exists.
|
|
|
- */
|
|
|
- if (corruption != null) {
|
|
|
- throw new IOException("Gap exists in logs from "
|
|
|
- + fromTxId, corruption);
|
|
|
- } else {
|
|
|
- return null;
|
|
|
- }
|
|
|
+ try {
|
|
|
+ editLogStream.writeRaw(data, 0, length);
|
|
|
+ } catch (IOException ex) {
|
|
|
+ // All journals have failed, it will be handled in logSync.
|
|
|
}
|
|
|
-
|
|
|
- return bestjm.getInputStream(fromTxId);
|
|
|
+ endTransaction(start);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Run recovery on all journals to recover any unclosed segments
|
|
|
*/
|
|
|
void recoverUnclosedStreams() {
|
|
|
- mapJournalsAndReportErrors(new JournalClosure() {
|
|
|
- @Override
|
|
|
- public void apply(JournalAndStream jas) throws IOException {
|
|
|
- jas.manager.recoverUnfinalizedSegments();
|
|
|
- }
|
|
|
- }, "recovering unclosed streams");
|
|
|
+ try {
|
|
|
+ journalSet.recoverUnfinalizedSegments();
|
|
|
+ } catch (IOException ex) {
|
|
|
+ // All journals have failed, it is handled in logSync.
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1143,23 +915,16 @@ public class FSEditLog {
|
|
|
* @param fromTxId first transaction in the selected streams
|
|
|
* @param toAtLeast the selected streams must contain this transaction
|
|
|
*/
|
|
|
- Collection<EditLogInputStream> selectInputStreams(long fromTxId, long toAtLeastTxId)
|
|
|
- throws IOException {
|
|
|
- List<EditLogInputStream> streams = Lists.newArrayList();
|
|
|
-
|
|
|
- boolean gapFound = false;
|
|
|
- EditLogInputStream stream = selectStream(fromTxId);
|
|
|
+ Collection<EditLogInputStream> selectInputStreams(long fromTxId,
|
|
|
+ long toAtLeastTxId) throws IOException {
|
|
|
+ List<EditLogInputStream> streams = new ArrayList<EditLogInputStream>();
|
|
|
+ EditLogInputStream stream = journalSet.getInputStream(fromTxId);
|
|
|
while (stream != null) {
|
|
|
fromTxId = stream.getLastTxId() + 1;
|
|
|
streams.add(stream);
|
|
|
- try {
|
|
|
- stream = selectStream(fromTxId);
|
|
|
- } catch (IOException ioe) {
|
|
|
- gapFound = true;
|
|
|
- break;
|
|
|
- }
|
|
|
+ stream = journalSet.getInputStream(fromTxId);
|
|
|
}
|
|
|
- if (fromTxId <= toAtLeastTxId || gapFound) {
|
|
|
+ if (fromTxId <= toAtLeastTxId) {
|
|
|
closeAllStreams(streams);
|
|
|
throw new IOException("No non-corrupt logs for txid "
|
|
|
+ fromTxId);
|
|
@@ -1176,75 +941,4 @@ public class FSEditLog {
|
|
|
IOUtils.closeStream(s);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * Container for a JournalManager paired with its currently
|
|
|
- * active stream.
|
|
|
- *
|
|
|
- * If a Journal gets disabled due to an error writing to its
|
|
|
- * stream, then the stream will be aborted and set to null.
|
|
|
- */
|
|
|
- static class JournalAndStream {
|
|
|
- private final JournalManager manager;
|
|
|
- private EditLogOutputStream stream;
|
|
|
- private long segmentStartsAtTxId = HdfsConstants.INVALID_TXID;
|
|
|
-
|
|
|
- private JournalAndStream(JournalManager manager) {
|
|
|
- this.manager = manager;
|
|
|
- }
|
|
|
-
|
|
|
- private void startLogSegment(long txId) throws IOException {
|
|
|
- Preconditions.checkState(stream == null);
|
|
|
- stream = manager.startLogSegment(txId);
|
|
|
- segmentStartsAtTxId = txId;
|
|
|
- }
|
|
|
-
|
|
|
- private void close(long lastTxId) throws IOException {
|
|
|
- Preconditions.checkArgument(lastTxId >= segmentStartsAtTxId,
|
|
|
- "invalid segment: lastTxId %s >= " +
|
|
|
- "segment starting txid %s", lastTxId, segmentStartsAtTxId);
|
|
|
-
|
|
|
- if (stream == null) return;
|
|
|
- stream.close();
|
|
|
- manager.finalizeLogSegment(segmentStartsAtTxId, lastTxId);
|
|
|
- stream = null;
|
|
|
- }
|
|
|
-
|
|
|
- @VisibleForTesting
|
|
|
- void abort() {
|
|
|
- if (stream == null) return;
|
|
|
- try {
|
|
|
- stream.abort();
|
|
|
- } catch (IOException ioe) {
|
|
|
- LOG.error("Unable to abort stream " + stream, ioe);
|
|
|
- }
|
|
|
- stream = null;
|
|
|
- segmentStartsAtTxId = HdfsConstants.INVALID_TXID;
|
|
|
- }
|
|
|
-
|
|
|
- private boolean isActive() {
|
|
|
- return stream != null;
|
|
|
- }
|
|
|
-
|
|
|
- @VisibleForTesting
|
|
|
- EditLogOutputStream getCurrentStream() {
|
|
|
- return stream;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public String toString() {
|
|
|
- return "JournalAndStream(mgr=" + manager +
|
|
|
- ", " + "stream=" + stream + ")";
|
|
|
- }
|
|
|
-
|
|
|
- @VisibleForTesting
|
|
|
- void setCurrentStreamForTests(EditLogOutputStream stream) {
|
|
|
- this.stream = stream;
|
|
|
- }
|
|
|
-
|
|
|
- @VisibleForTesting
|
|
|
- JournalManager getManager() {
|
|
|
- return manager;
|
|
|
- }
|
|
|
- }
|
|
|
}
|