HDFS-13609. [SBN read] Edit Tail Fast Path Part 3: NameNode-side changes to support tailing edits via RPC. Contributed by Erik Krogen.

HDFS-10519. Add a configuration option to enable in-progress edit log tailing. Contributed by Jiayi Zhou.
This commit is contained in:
Erik Krogen 2018-05-22 16:45:26 -07:00 committed by Chen Liang
parent 88d65af8a1
commit 85f81fe458
32 changed files with 964 additions and 70 deletions

View File

@ -297,7 +297,19 @@ public class IOUtils {
cleanupWithLogger(null, stream);
}
}
/**
* Closes the streams ignoring {@link Throwable}.
* Must only be called in cleaning up from exception handlers.
*
* @param streams the Streams to close
*/
public static void closeStreams(java.io.Closeable... streams) {
if (streams != null) {
cleanupWithLogger(null, streams);
}
}
/**
* Closes the socket ignoring {@link IOException}
*

View File

@ -533,12 +533,18 @@ public class BookKeeperJournalManager implements JournalManager {
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
throw new IOException("Error finalising ledger", ie);
}
}
}
public void selectInputStreams(
Collection<EditLogInputStream> streams,
long fromTxnId, boolean inProgressOk) throws IOException {
selectInputStreams(streams, fromTxnId, inProgressOk, false);
}
@Override
public void selectInputStreams(Collection<EditLogInputStream> streams,
long fromTxId, boolean inProgressOk)
long fromTxId, boolean inProgressOk, boolean onlyDurableTxns)
throws IOException {
List<EditLogLedgerMetadata> currentLedgerList = getLedgerList(fromTxId,
inProgressOk);

View File

@ -277,7 +277,8 @@ public class PBHelper {
public static RemoteEditLogManifestProto convert(
RemoteEditLogManifest manifest) {
RemoteEditLogManifestProto.Builder builder = RemoteEditLogManifestProto
.newBuilder();
.newBuilder()
.setCommittedTxnId(manifest.getCommittedTxnId());
for (RemoteEditLog log : manifest.getLogs()) {
builder.addLogs(convert(log));
}
@ -291,7 +292,8 @@ public class PBHelper {
for (RemoteEditLogProto l : manifest.getLogsList()) {
logs.add(convert(l));
}
return new RemoteEditLogManifest(logs);
return new RemoteEditLogManifest(logs,
manifest.getCommittedTxnId());
}
public static CheckpointCommandProto convert(CheckpointCommand cmd) {

View File

@ -22,6 +22,7 @@ import java.net.URL;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
@ -106,6 +107,12 @@ interface AsyncLogger {
* Begin a new epoch on the target node.
*/
public ListenableFuture<NewEpochResponseProto> newEpoch(long epoch);
/**
* Fetch journaled edits from the cache.
*/
public ListenableFuture<GetJournaledEditsResponseProto> getJournaledEdits(
long fromTxnId, int maxTransactions);
/**
* Fetch the list of edit logs available on the remote node.

View File

@ -26,6 +26,7 @@ import java.util.concurrent.TimeoutException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
@ -261,6 +262,19 @@ class AsyncLoggerSet {
return QuorumCall.create(calls);
}
public QuorumCall<AsyncLogger, GetJournaledEditsResponseProto>
getJournaledEdits(long fromTxnId, int maxTransactions) {
Map<AsyncLogger,
ListenableFuture<GetJournaledEditsResponseProto>> calls
= Maps.newHashMap();
for (AsyncLogger logger : loggers) {
ListenableFuture<GetJournaledEditsResponseProto> future =
logger.getJournaledEdits(fromTxnId, maxTransactions);
calls.put(logger, future);
}
return QuorumCall.create(calls);
}
public QuorumCall<AsyncLogger, RemoteEditLogManifest> getEditLogManifest(
long fromTxnId, boolean inProgressOk) {
Map<AsyncLogger,

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
@ -547,6 +548,19 @@ public class IPCLoggerChannel implements AsyncLogger {
});
}
@Override
public ListenableFuture<GetJournaledEditsResponseProto> getJournaledEdits(
long fromTxnId, int maxTransactions) {
return parallelExecutor.submit(
new Callable<GetJournaledEditsResponseProto>() {
@Override
public GetJournaledEditsResponseProto call() throws IOException {
return getProxy().getJournaledEdits(journalId, nameServiceId,
fromTxnId, maxTransactions);
}
});
}
@Override
public ListenableFuture<RemoteEditLogManifest> getEditLogManifest(
final long fromTxnId, final boolean inProgressOk) {

View File

@ -22,6 +22,7 @@ import java.net.InetSocketAddress;
import java.net.URI;
import java.net.URL;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
@ -37,6 +38,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
@ -69,6 +71,14 @@ import com.google.protobuf.TextFormat;
public class QuorumJournalManager implements JournalManager {
static final Log LOG = LogFactory.getLog(QuorumJournalManager.class);
// This config is not publicly exposed
static final String QJM_RPC_MAX_TXNS_KEY =
"dfs.ha.tail-edits.qjm.rpc.max-txns";
static final int QJM_RPC_MAX_TXNS_DEFAULT = 5000;
// Maximum number of transactions to fetch at a time when using the
// RPC edit fetch mechanism
private final int maxTxnsPerRpc;
// Timeouts for which the QJM will wait for each of the following actions.
private final int startSegmentTimeoutMs;
private final int prepareRecoveryTimeoutMs;
@ -111,6 +121,10 @@ public class QuorumJournalManager implements JournalManager {
this.connectionFactory = URLConnectionFactory
.newDefaultURLConnectionFactory(conf);
this.maxTxnsPerRpc =
conf.getInt(QJM_RPC_MAX_TXNS_KEY, QJM_RPC_MAX_TXNS_DEFAULT);
Preconditions.checkArgument(maxTxnsPerRpc > 0,
"Must specify %s greater than 0!", QJM_RPC_MAX_TXNS_KEY);
// Configure timeouts.
this.startSegmentTimeoutMs = conf.getInt(
DFSConfigKeys.DFS_QJOURNAL_START_SEGMENT_TIMEOUT_KEY,
@ -404,8 +418,11 @@ public class QuorumJournalManager implements JournalManager {
layoutVersion);
loggers.waitForWriteQuorum(q, startSegmentTimeoutMs,
"startLogSegment(" + txId + ")");
return new QuorumOutputStream(loggers, txId,
outputBufferCapacity, writeTxnsTimeoutMs);
boolean updateCommittedTxId = conf.getBoolean(
DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY,
DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_DEFAULT);
return new QuorumOutputStream(loggers, txId, outputBufferCapacity,
writeTxnsTimeoutMs, updateCommittedTxId);
}
@Override
@ -464,32 +481,134 @@ public class QuorumJournalManager implements JournalManager {
loggers.close();
}
@Override
public void selectInputStreams(Collection<EditLogInputStream> streams,
long fromTxnId, boolean inProgressOk) throws IOException {
selectInputStreams(streams, fromTxnId, inProgressOk, false);
}
@Override
public void selectInputStreams(Collection<EditLogInputStream> streams,
long fromTxnId, boolean inProgressOk,
boolean onlyDurableTxns) throws IOException {
if (inProgressOk) {
LOG.info("Tailing edits starting from txn ID " + fromTxnId +
" via RPC mechanism");
try {
Collection<EditLogInputStream> rpcStreams = new ArrayList<>();
selectRpcInputStreams(rpcStreams, fromTxnId, onlyDurableTxns);
streams.addAll(rpcStreams);
return;
} catch (IOException ioe) {
LOG.warn("Encountered exception while tailing edits >= " + fromTxnId +
" via RPC; falling back to streaming.", ioe);
}
}
selectStreamingInputStreams(streams, fromTxnId, inProgressOk,
onlyDurableTxns);
}
/**
* Select input streams from the journals, specifically using the RPC
* mechanism optimized for low latency.
*
* @param streams The collection to store the return streams into.
* @param fromTxnId Select edits starting from this transaction ID
* @param onlyDurableTxns Iff true, only include transactions which have been
* committed to a quorum of the journals.
* @throws IOException Upon issues, including cache misses on the journals.
*/
private void selectRpcInputStreams(Collection<EditLogInputStream> streams,
long fromTxnId, boolean onlyDurableTxns) throws IOException {
QuorumCall<AsyncLogger, GetJournaledEditsResponseProto> q =
loggers.getJournaledEdits(fromTxnId, maxTxnsPerRpc);
Map<AsyncLogger, GetJournaledEditsResponseProto> responseMap =
loggers.waitForWriteQuorum(q, selectInputStreamsTimeoutMs,
"selectRpcInputStreams");
assert responseMap.size() >= loggers.getMajoritySize() :
"Quorum call returned without a majority";
List<Integer> responseCounts = new ArrayList<>();
for (GetJournaledEditsResponseProto resp : responseMap.values()) {
responseCounts.add(resp.getTxnCount());
}
Collections.sort(responseCounts);
int highestTxnCount = responseCounts.get(responseCounts.size() - 1);
if (LOG.isDebugEnabled() || highestTxnCount < 0) {
StringBuilder msg = new StringBuilder("Requested edits starting from ");
msg.append(fromTxnId).append("; got ").append(responseMap.size())
.append(" responses: <");
for (Map.Entry<AsyncLogger, GetJournaledEditsResponseProto> ent :
responseMap.entrySet()) {
msg.append("[").append(ent.getKey()).append(", ")
.append(ent.getValue().getTxnCount()).append("],");
}
msg.append(">");
if (highestTxnCount < 0) {
throw new IOException("Did not get any valid JournaledEdits " +
"responses: " + msg);
} else {
LOG.debug(msg.toString());
}
}
int maxAllowedTxns = !onlyDurableTxns ? highestTxnCount :
responseCounts.get(responseCounts.size() - loggers.getMajoritySize());
if (maxAllowedTxns == 0) {
LOG.debug("No new edits available in logs; requested starting from " +
"ID " + fromTxnId);
return;
}
LOG.info("Selected loggers with >= " + maxAllowedTxns +
" transactions starting from " + fromTxnId);
PriorityQueue<EditLogInputStream> allStreams = new PriorityQueue<>(
JournalSet.EDIT_LOG_INPUT_STREAM_COMPARATOR);
for (GetJournaledEditsResponseProto resp : responseMap.values()) {
long endTxnId = fromTxnId - 1 +
Math.min(maxAllowedTxns, resp.getTxnCount());
allStreams.add(EditLogFileInputStream.fromByteString(
resp.getEditLog(), fromTxnId, endTxnId, true));
}
JournalSet.chainAndMakeRedundantStreams(streams, allStreams, fromTxnId);
}
/**
* Select input streams from the journals, specifically using the streaming
* mechanism optimized for resiliency / bulk load.
*/
private void selectStreamingInputStreams(
Collection<EditLogInputStream> streams, long fromTxnId,
boolean inProgressOk, boolean onlyDurableTxns) throws IOException {
QuorumCall<AsyncLogger, RemoteEditLogManifest> q =
loggers.getEditLogManifest(fromTxnId, inProgressOk);
Map<AsyncLogger, RemoteEditLogManifest> resps =
loggers.waitForWriteQuorum(q, selectInputStreamsTimeoutMs,
"selectInputStreams");
LOG.debug("selectInputStream manifests:\n" +
"selectStreamingInputStreams");
LOG.debug("selectStreamingInputStream manifests:\n" +
Joiner.on("\n").withKeyValueSeparator(": ").join(resps));
final PriorityQueue<EditLogInputStream> allStreams =
final PriorityQueue<EditLogInputStream> allStreams =
new PriorityQueue<EditLogInputStream>(64,
JournalSet.EDIT_LOG_INPUT_STREAM_COMPARATOR);
for (Map.Entry<AsyncLogger, RemoteEditLogManifest> e : resps.entrySet()) {
AsyncLogger logger = e.getKey();
RemoteEditLogManifest manifest = e.getValue();
long committedTxnId = manifest.getCommittedTxnId();
for (RemoteEditLog remoteLog : manifest.getLogs()) {
URL url = logger.buildURLToFetchLogs(remoteLog.getStartTxId());
long endTxId = remoteLog.getEndTxId();
// If it's bounded by durable Txns, endTxId could not be larger
// than committedTxnId. This ensures the consistency.
if (onlyDurableTxns && inProgressOk) {
endTxId = Math.min(endTxId, committedTxnId);
}
EditLogInputStream elis = EditLogFileInputStream.fromUrl(
connectionFactory, url, remoteLog.getStartTxId(),
remoteLog.getEndTxId(), remoteLog.isInProgress());
endTxId, remoteLog.isInProgress());
allStreams.add(elis);
}
}

View File

@ -33,15 +33,17 @@ class QuorumOutputStream extends EditLogOutputStream {
private EditsDoubleBuffer buf;
private final long segmentTxId;
private final int writeTimeoutMs;
private final boolean updateCommittedTxId;
public QuorumOutputStream(AsyncLoggerSet loggers,
long txId, int outputBufferCapacity,
int writeTimeoutMs) throws IOException {
int writeTimeoutMs, boolean updateCommittedTxId) throws IOException {
super();
this.buf = new EditsDoubleBuffer(outputBufferCapacity);
this.loggers = loggers;
this.segmentTxId = txId;
this.writeTimeoutMs = writeTimeoutMs;
this.updateCommittedTxId = updateCommittedTxId;
}
@Override
@ -110,6 +112,15 @@ class QuorumOutputStream extends EditLogOutputStream {
// RPCs will thus let the loggers know of the most recent transaction, even
// if a logger has fallen behind.
loggers.setCommittedTxId(firstTxToFlush + numReadyTxns - 1);
// If we don't have this dummy send, committed TxId might be one-batch
// stale on the Journal Nodes
if (updateCommittedTxId) {
QuorumCall<AsyncLogger, Void> fakeCall = loggers.sendEdits(
segmentTxId, firstTxToFlush,
0, new byte[0]);
loggers.waitForWriteQuorum(fakeCall, writeTimeoutMs, "sendEdits");
}
}
}

View File

@ -268,8 +268,8 @@ public class Journal implements Closeable {
checkFormatted();
return lastWriterEpoch.get();
}
synchronized long getCommittedTxnIdForTests() throws IOException {
synchronized long getCommittedTxnId() throws IOException {
return committedTxnId.get();
}
@ -375,6 +375,12 @@ public class Journal implements Closeable {
checkFormatted();
checkWriteRequest(reqInfo);
// If numTxns is 0, it's actually a fake send which aims at updating
// committedTxId only. So we can return early.
if (numTxns == 0) {
return;
}
checkSync(curSegment != null,
"Can't write, no segment open" + " ; journal id: " + journalId);
if (curSegmentTxId != segmentTxId) {
@ -704,12 +710,12 @@ public class Journal implements Closeable {
}
}
if (log != null && log.isInProgress()) {
logs.add(new RemoteEditLog(log.getStartTxId(), getHighestWrittenTxId(),
true));
logs.add(new RemoteEditLog(log.getStartTxId(),
getHighestWrittenTxId(), true));
}
}
return new RemoteEditLogManifest(logs);
return new RemoteEditLogManifest(logs, getCommittedTxnId());
}
/**

View File

@ -80,7 +80,7 @@ class BackupJournalManager implements JournalManager {
@Override
public void selectInputStreams(Collection<EditLogInputStream> streams,
long fromTxnId, boolean inProgressOk) {
long fromTxnId, boolean inProgressOk, boolean onlyDurableTxns) {
// This JournalManager is never used for input. Therefore it cannot
// return any transactions
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hdfs.server.namenode;
import com.google.protobuf.ByteString;
import java.io.BufferedInputStream;
import java.io.DataInputStream;
import java.io.EOFException;
@ -120,6 +121,23 @@ public class EditLogFileInputStream extends EditLogInputStream {
return new EditLogFileInputStream(new URLLog(connectionFactory, url),
startTxId, endTxId, inProgress);
}
/**
* Create an EditLogInputStream from a {@link ByteString}, i.e. an in-memory
* collection of bytes.
*
* @param bytes The byte string to read from
* @param startTxId the expected starting transaction ID
* @param endTxId the expected ending transaction ID
* @param inProgress whether the log is in-progress
* @return An edit stream to read from
*/
public static EditLogInputStream fromByteString(ByteString bytes,
long startTxId, long endTxId, boolean inProgress) {
return new EditLogFileInputStream(new ByteStringLog(bytes,
String.format("ByteStringEditLog[%d, %d]", startTxId, endTxId)),
startTxId, endTxId, inProgress);
}
private EditLogFileInputStream(LogSource log,
long firstTxId, long lastTxId,
@ -377,6 +395,32 @@ public class EditLogFileInputStream extends EditLogInputStream {
public long length();
public String getName();
}
private static class ByteStringLog implements LogSource {
private final ByteString bytes;
private final String name;
public ByteStringLog(ByteString bytes, String name) {
this.bytes = bytes;
this.name = name;
}
@Override
public InputStream getInputStream() {
return bytes.newInput();
}
@Override
public long length() {
return bytes.size();
}
@Override
public String getName() {
return name;
}
}
private static class FileLog implements LogSource {
private final File file;

View File

@ -320,7 +320,7 @@ public class FSEditLog implements LogsPurgeable {
// Safety check: we should never start a segment if there are
// newer txids readable.
List<EditLogInputStream> streams = new ArrayList<EditLogInputStream>();
journalSet.selectInputStreams(streams, segmentTxId, true);
journalSet.selectInputStreams(streams, segmentTxId, true, false);
if (!streams.isEmpty()) {
String error = String.format("Cannot start writing at txid %s " +
"when there is a stream available for read: %s",
@ -1586,15 +1586,23 @@ public class FSEditLog implements LogsPurgeable {
@Override
public void selectInputStreams(Collection<EditLogInputStream> streams,
long fromTxId, boolean inProgressOk) throws IOException {
journalSet.selectInputStreams(streams, fromTxId, inProgressOk);
long fromTxId, boolean inProgressOk, boolean onlyDurableTxns)
throws IOException {
journalSet.selectInputStreams(streams, fromTxId,
inProgressOk, onlyDurableTxns);
}
public Collection<EditLogInputStream> selectInputStreams(
long fromTxId, long toAtLeastTxId) throws IOException {
return selectInputStreams(fromTxId, toAtLeastTxId, null, true);
return selectInputStreams(fromTxId, toAtLeastTxId, null, true, false);
}
public Collection<EditLogInputStream> selectInputStreams(
long fromTxId, long toAtLeastTxId, MetaRecoveryContext recovery,
boolean inProgressOK) throws IOException {
return selectInputStreams(fromTxId, toAtLeastTxId,
recovery, inProgressOK, false);
}
/**
* Select a list of input streams.
*
@ -1602,16 +1610,18 @@ public class FSEditLog implements LogsPurgeable {
* @param toAtLeastTxId the selected streams must contain this transaction
* @param recovery recovery context
* @param inProgressOk set to true if in-progress streams are OK
* @param onlyDurableTxns set to true if streams are bounded
* by the durable TxId
*/
public Collection<EditLogInputStream> selectInputStreams(
long fromTxId, long toAtLeastTxId, MetaRecoveryContext recovery,
boolean inProgressOk) throws IOException {
public Collection<EditLogInputStream> selectInputStreams(long fromTxId,
long toAtLeastTxId, MetaRecoveryContext recovery, boolean inProgressOk,
boolean onlyDurableTxns) throws IOException {
List<EditLogInputStream> streams = new ArrayList<EditLogInputStream>();
synchronized(journalSetLock) {
Preconditions.checkState(journalSet.isOpen(), "Cannot call " +
"selectInputStreams() on closed FSEditLog");
selectInputStreams(streams, fromTxId, inProgressOk);
selectInputStreams(streams, fromTxId, inProgressOk, onlyDurableTxns);
}
try {

View File

@ -334,10 +334,17 @@ public class FileJournalManager implements JournalManager {
return ret;
}
synchronized public void selectInputStreams(
Collection<EditLogInputStream> streams,
long fromTxnId, boolean inProgressOk) throws IOException {
selectInputStreams(streams, fromTxnId, inProgressOk, false);
}
@Override
synchronized public void selectInputStreams(
Collection<EditLogInputStream> streams, long fromTxId,
boolean inProgressOk) throws IOException {
boolean inProgressOk, boolean onlyDurableTxns)
throws IOException {
List<EditLogFile> elfs = matchEditLogs(sd.getCurrentDir());
if (LOG.isDebugEnabled()) {
LOG.debug(this + ": selecting input streams starting at " + fromTxId +

View File

@ -262,10 +262,13 @@ public class JournalSet implements JournalManager {
* may not be sorted-- this is up to the caller.
* @param fromTxId The transaction ID to start looking for streams at
* @param inProgressOk Should we consider unfinalized streams?
* @param onlyDurableTxns Set to true if streams are bounded by the durable
* TxId. A durable TxId is the committed txid in QJM
* or the largest txid written into file in FJM
*/
@Override
public void selectInputStreams(Collection<EditLogInputStream> streams,
long fromTxId, boolean inProgressOk) throws IOException {
long fromTxId, boolean inProgressOk, boolean onlyDurableTxns) throws IOException {
final PriorityQueue<EditLogInputStream> allStreams =
new PriorityQueue<EditLogInputStream>(64,
EDIT_LOG_INPUT_STREAM_COMPARATOR);
@ -275,7 +278,8 @@ public class JournalSet implements JournalManager {
continue;
}
try {
jas.getManager().selectInputStreams(allStreams, fromTxId, inProgressOk);
jas.getManager().selectInputStreams(allStreams, fromTxId,
inProgressOk, onlyDurableTxns);
} catch (IOException ioe) {
LOG.warn("Unable to determine input streams from " + jas.getManager() +
". Skipping.", ioe);
@ -682,7 +686,8 @@ public class JournalSet implements JournalManager {
// And then start looking from after that point
curStartTxId = bestLog.getEndTxId() + 1;
}
RemoteEditLogManifest ret = new RemoteEditLogManifest(logs);
RemoteEditLogManifest ret = new RemoteEditLogManifest(logs,
curStartTxId - 1);
if (LOG.isDebugEnabled()) {
LOG.debug("Generated manifest for logs since " + fromTxId + ":"

View File

@ -42,10 +42,14 @@ interface LogsPurgeable {
*
* @param fromTxId the first transaction id we want to read
* @param inProgressOk whether or not in-progress streams should be returned
* @param onlyDurableTxns whether or not streams should be bounded by durable
* TxId. A durable TxId is the committed txid in QJM
* or the largest txid written into file in FJM
* @throws IOException if the underlying storage has an error or is otherwise
* inaccessible
*/
void selectInputStreams(Collection<EditLogInputStream> streams,
long fromTxId, boolean inProgressOk) throws IOException;
long fromTxId, boolean inProgressOk, boolean onlyDurableTxns)
throws IOException;
}

View File

@ -134,7 +134,7 @@ public class NNStorageRetentionManager {
long purgeLogsFrom = Math.max(0, minimumRequiredTxId - numExtraEditsToRetain);
ArrayList<EditLogInputStream> editLogs = new ArrayList<EditLogInputStream>();
purgeableLogs.selectInputStreams(editLogs, purgeLogsFrom, false);
purgeableLogs.selectInputStreams(editLogs, purgeLogsFrom, false, false);
Collections.sort(editLogs, new Comparator<EditLogInputStream>() {
@Override
public int compare(EditLogInputStream a, EditLogInputStream b) {

View File

@ -914,7 +914,7 @@ public class SecondaryNameNode implements Runnable,
@Override
public void selectInputStreams(Collection<EditLogInputStream> streams,
long fromTxId, boolean inProgressOk) {
long fromTxId, boolean inProgressOk, boolean onlyDurableTxns) {
Iterator<StorageDirectory> iter = storage.dirIterator();
while (iter.hasNext()) {
StorageDirectory dir = iter.next();

View File

@ -135,6 +135,11 @@ public class EditLogTailer {
*/
private int maxRetries;
/**
* Whether the tailer should tail the in-progress edit log segments.
*/
private final boolean inProgressOk;
public EditLogTailer(FSNamesystem namesystem, Configuration conf) {
this.tailerThread = new EditLogTailerThread();
this.conf = conf;
@ -182,6 +187,10 @@ public class EditLogTailer {
maxRetries = DFSConfigKeys.DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_DEFAULT;
}
inProgressOk = conf.getBoolean(
DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY,
DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_DEFAULT);
nnCount = nns.size();
// setup the iterator to endlessly loop the nns
this.nnLookup = Iterators.cycle(nns);
@ -263,7 +272,8 @@ public class EditLogTailer {
}
Collection<EditLogInputStream> streams;
try {
streams = editLog.selectInputStreams(lastTxnId + 1, 0, null, false);
streams = editLog.selectInputStreams(lastTxnId + 1, 0,
null, inProgressOk, true);
} catch (IOException ioe) {
// This is acceptable. If we try to tail edits in the middle of an edits
// log roll, i.e. the last one has been finalized but the new inprogress

View File

@ -29,12 +29,15 @@ import com.google.common.base.Preconditions;
public class RemoteEditLogManifest {
private List<RemoteEditLog> logs;
private long committedTxnId = -1;
public RemoteEditLogManifest() {
}
public RemoteEditLogManifest(List<RemoteEditLog> logs) {
public RemoteEditLogManifest(List<RemoteEditLog> logs, long committedTxnId) {
this.logs = logs;
this.committedTxnId = committedTxnId;
checkState();
}
@ -46,7 +49,7 @@ public class RemoteEditLogManifest {
*/
private void checkState() {
Preconditions.checkNotNull(logs);
RemoteEditLog prev = null;
for (RemoteEditLog log : logs) {
if (prev != null) {
@ -56,7 +59,6 @@ public class RemoteEditLogManifest {
+ this);
}
}
prev = log;
}
}
@ -65,10 +67,13 @@ public class RemoteEditLogManifest {
return Collections.unmodifiableList(logs);
}
public long getCommittedTxnId() {
return committedTxnId;
}
@Override
public String toString() {
return "[" + Joiner.on(", ").join(logs) + "]";
return "[" + Joiner.on(", ").join(logs) + "]" + " CommittedTxId: "
+ committedTxnId;
}
}

View File

@ -92,6 +92,7 @@ message RemoteEditLogProto {
*/
message RemoteEditLogManifestProto {
repeated RemoteEditLogProto logs = 1;
required uint64 committedTxnId = 2;
}
/**

View File

@ -2793,6 +2793,16 @@
</description>
</property>
<property>
<name>dfs.ha.tail-edits.in-progress</name>
<value>false</value>
<description>
Whether enable standby namenode to tail in-progress edit logs.
Clients might want to turn it on when they want Standby NN to have
more up-to-date data.
</description>
</property>
<property>
<name>dfs.namenode.lease-recheck-interval-ms</name>
<value>2000</value>

View File

@ -308,7 +308,7 @@ public class TestPBHelper {
List<RemoteEditLog> logs = new ArrayList<RemoteEditLog>();
logs.add(new RemoteEditLog(1, 10));
logs.add(new RemoteEditLog(11, 20));
RemoteEditLogManifest m = new RemoteEditLogManifest(logs);
RemoteEditLogManifest m = new RemoteEditLogManifest(logs, 20);
RemoteEditLogManifestProto mProto = PBHelper.convert(m);
RemoteEditLogManifest m1 = PBHelper.convert(mProto);

View File

@ -44,6 +44,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
import org.apache.hadoop.hdfs.qjournal.QJMTestUtil;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
@ -92,6 +93,10 @@ public class TestQuorumJournalManager {
conf = new Configuration();
// Don't retry connections - it just slows down the tests.
conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
// Turn off IPC client caching to handle daemon restarts.
conf.setInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, 0);
conf.setBoolean(DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY, true);
cluster = new MiniJournalCluster.Builder(conf)
.baseDir(GenericTestUtils.getRandomizedTestDir().getAbsolutePath())
@ -935,7 +940,156 @@ public class TestQuorumJournalManager {
verifyEdits(streams, 25, 50);
}
@Test
public void testInProgressRecovery() throws Exception {
// Test the case when in-progress edit log tailing is on, and
// new active performs recovery when the old active crashes
// without closing the last log segment.
// See HDFS-13145 for more details.
// Write two batches of edits. After these, the commitId on the
// journals should be 5, and endTxnId should be 8.
EditLogOutputStream stm = qjm.startLogSegment(1,
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
writeTxns(stm, 1, 5);
writeTxns(stm, 6, 3);
// Do recovery from a separate QJM, just like in failover.
QuorumJournalManager qjm2 = createSpyingQJM();
qjm2.recoverUnfinalizedSegments();
checkRecovery(cluster, 1, 8);
// When selecting input stream, we should see all txns up to 8.
List<EditLogInputStream> streams = new ArrayList<>();
qjm2.selectInputStreams(streams, 1, true, true);
verifyEdits(streams, 1, 8);
}
@Test
public void testSelectViaRpcWithDurableTransactions() throws Exception {
// Two loggers will have up to ID 5, one will have up to ID 6
failLoggerAtTxn(spies.get(0), 6);
failLoggerAtTxn(spies.get(1), 6);
EditLogOutputStream stm =
qjm.startLogSegment(1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
writeTxns(stm, 1, 5);
try {
writeTxns(stm, 6, 1);
fail("Did not fail to write when only a minority succeeded");
} catch (QuorumException qe) {
GenericTestUtils.assertExceptionContains(
"too many exceptions to achieve quorum size 2/3", qe);
}
List<EditLogInputStream> streams = new ArrayList<>();
qjm.selectInputStreams(streams, 1, true, true);
verifyEdits(streams, 1, 5);
IOUtils.closeStreams(streams.toArray(new Closeable[0]));
for (AsyncLogger logger : spies) {
Mockito.verify(logger, Mockito.times(1)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
}
}
@Test
public void testSelectViaRpcWithoutDurableTransactions() throws Exception {
setupLoggers345();
futureThrows(new IOException()).when(spies.get(1)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
List<EditLogInputStream> streams = new ArrayList<>();
qjm.selectInputStreams(streams, 1, true, false);
verifyEdits(streams, 1, 5);
IOUtils.closeStreams(streams.toArray(new Closeable[0]));
for (AsyncLogger logger : spies) {
Mockito.verify(logger, Mockito.times(1)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
}
}
@Test
public void testSelectViaRpcOneDeadJN() throws Exception {
EditLogOutputStream stm =
qjm.startLogSegment(1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
writeTxns(stm, 1, 10);
cluster.getJournalNode(0).stopAndJoin(0);
List<EditLogInputStream> streams = new ArrayList<>();
qjm.selectInputStreams(streams, 1, true, false);
verifyEdits(streams, 1, 10);
IOUtils.closeStreams(streams.toArray(new Closeable[0]));
}
@Test
public void testSelectViaRpcTwoDeadJNs() throws Exception {
EditLogOutputStream stm =
qjm.startLogSegment(1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
writeTxns(stm, 1, 10);
cluster.getJournalNode(0).stopAndJoin(0);
cluster.getJournalNode(1).stopAndJoin(0);
try {
qjm.selectInputStreams(new ArrayList<>(), 1, true, false);
fail("");
} catch (QuorumException qe) {
GenericTestUtils.assertExceptionContains(
"too many exceptions to achieve quorum size 2/3", qe);
}
}
@Test
public void testSelectViaRpcTwoJNsError() throws Exception {
EditLogOutputStream stm =
qjm.startLogSegment(1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
writeTxns(stm, 1, 10);
writeTxns(stm, 11, 1);
futureThrows(new IOException()).when(spies.get(0)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
futureThrows(new IOException()).when(spies.get(1)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
List<EditLogInputStream> streams = new ArrayList<>();
qjm.selectInputStreams(streams, 1, true, true);
// This should still succeed as the QJM should fall back to the streaming
// mechanism for fetching edits
verifyEdits(streams, 1, 11);
IOUtils.closeStreams(streams.toArray(new Closeable[0]));
for (AsyncLogger logger : spies) {
Mockito.verify(logger, Mockito.times(1)).getEditLogManifest(1, true);
}
}
@Test
public void testSelectViaRpcAfterJNRestart() throws Exception {
EditLogOutputStream stm =
qjm.startLogSegment(1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
writeTxns(stm, 1, 10);
qjm.finalizeLogSegment(1, 10);
// Close to avoid connections hanging around after the JNs are restarted
for (int i = 0; i < cluster.getNumNodes(); i++) {
cluster.restartJournalNode(i);
}
cluster.waitActive();
qjm = createSpyingQJM();
spies = qjm.getLoggerSetForTests().getLoggersForTests();
List<EditLogInputStream> streams = new ArrayList<>();
qjm.selectInputStreams(streams, 1, true, true);
// This should still succeed as the QJM should fall back to the streaming
// mechanism for fetching edits
verifyEdits(streams, 1, 10);
IOUtils.closeStreams(streams.toArray(new Closeable[0]));
for (AsyncLogger logger : spies) {
Mockito.verify(logger, Mockito.times(1)).getEditLogManifest(1, true);
}
}
private QuorumJournalManager createSpyingQJM()
throws IOException, URISyntaxException {

View File

@ -17,10 +17,13 @@
*/
package org.apache.hadoop.hdfs.qjournal.client;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.anyLong;
import static org.mockito.Matchers.eq;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.net.URI;
import java.util.List;
@ -28,11 +31,11 @@ import java.util.List;
import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.qjournal.client.AsyncLogger;
import org.apache.hadoop.hdfs.qjournal.client.QuorumException;
import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@ -44,11 +47,15 @@ import org.mockito.Mockito;
import org.mockito.stubbing.Stubber;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
import com.google.protobuf.ByteString;
import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.writeOp;
import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.createTxnData;
import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.verifyEdits;
/**
* True unit tests for QuorumJournalManager
@ -215,6 +222,94 @@ public class TestQuorumJournalManagerUnit {
Mockito.verify(spyLoggers.get(0)).setCommittedTxId(1L);
}
@Test
public void testReadRpcInputStreams() throws Exception {
for (int jn = 0; jn < 3; jn++) {
futureReturns(getJournaledEditsReponse(1, 3))
.when(spyLoggers.get(jn)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
}
List<EditLogInputStream> streams = Lists.newArrayList();
qjm.selectInputStreams(streams, 1, true, true);
assertEquals(1, streams.size());
verifyEdits(streams, 1, 3);
}
@Test
public void testReadRpcMismatchedInputStreams() throws Exception {
for (int jn = 0; jn < 3; jn++) {
futureReturns(getJournaledEditsReponse(1, jn + 1))
.when(spyLoggers.get(jn)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
}
List<EditLogInputStream> streams = Lists.newArrayList();
qjm.selectInputStreams(streams, 1, true, true);
assertEquals(1, streams.size());
verifyEdits(streams, 1, 2);
}
@Test
public void testReadRpcInputStreamsOneSlow() throws Exception {
for (int jn = 0; jn < 2; jn++) {
futureReturns(getJournaledEditsReponse(1, jn + 1))
.when(spyLoggers.get(jn)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
}
Mockito.doReturn(SettableFuture.create())
.when(spyLoggers.get(2)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
List<EditLogInputStream> streams = Lists.newArrayList();
qjm.selectInputStreams(streams, 1, true, true);
assertEquals(1, streams.size());
verifyEdits(streams, 1, 1);
}
@Test
public void testReadRpcInputStreamsOneException() throws Exception {
for (int jn = 0; jn < 2; jn++) {
futureReturns(getJournaledEditsReponse(1, jn + 1))
.when(spyLoggers.get(jn)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
}
futureThrows(new IOException()).when(spyLoggers.get(2))
.getJournaledEdits(1, QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
List<EditLogInputStream> streams = Lists.newArrayList();
qjm.selectInputStreams(streams, 1, true, true);
assertEquals(1, streams.size());
verifyEdits(streams, 1, 1);
}
@Test
public void testReadRpcInputStreamsNoNewEdits() throws Exception {
for (int jn = 0; jn < 3; jn++) {
futureReturns(GetJournaledEditsResponseProto.newBuilder()
.setTxnCount(0).setEditLog(ByteString.EMPTY).build())
.when(spyLoggers.get(jn))
.getJournaledEdits(1, QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
}
List<EditLogInputStream> streams = Lists.newArrayList();
qjm.selectInputStreams(streams, 1, true, true);
assertEquals(0, streams.size());
}
private GetJournaledEditsResponseProto getJournaledEditsReponse(
int startTxn, int numTxns) throws Exception {
ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
EditLogFileOutputStream.writeHeader(
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION,
new DataOutputStream(byteStream));
byteStream.write(createTxnData(startTxn, numTxns));
return GetJournaledEditsResponseProto.newBuilder()
.setTxnCount(numTxns)
.setEditLog(ByteString.copyFrom(byteStream.toByteArray()))
.build();
}
private EditLogOutputStream createLogSegment() throws IOException {
futureReturns(null).when(spyLoggers.get(0)).startLogSegment(Mockito.anyLong(),
Mockito.eq(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION));

View File

@ -167,12 +167,12 @@ public class TestJournal {
// Send txids 1-3, with a request indicating only 0 committed
journal.journal(new RequestInfo(JID, 1, 2, 0), 1, 1, 3,
QJMTestUtil.createTxnData(1, 3));
assertEquals(0, journal.getCommittedTxnIdForTests());
assertEquals(0, journal.getCommittedTxnId());
// Send 4-6, with request indicating that through 3 is committed.
journal.journal(new RequestInfo(JID, 1, 3, 3), 1, 4, 3,
QJMTestUtil.createTxnData(4, 6));
assertEquals(3, journal.getCommittedTxnIdForTests());
assertEquals(3, journal.getCommittedTxnId());
}
@Test (timeout = 10000)

View File

@ -1037,9 +1037,9 @@ public class TestEditLog {
"[1,100]|[101,200]|[201,]");
log = getFSEditLog(storage);
log.initJournalsForWrite();
assertEquals("[[1,100], [101,200]]",
assertEquals("[[1,100], [101,200]] CommittedTxId: 200",
log.getEditLogManifest(1).toString());
assertEquals("[[101,200]]",
assertEquals("[[101,200]] CommittedTxId: 200",
log.getEditLogManifest(101).toString());
// Another simple case, different directories have different
@ -1049,8 +1049,8 @@ public class TestEditLog {
"[1,100]|[201,300]|[301,400]"); // nothing starting at 101
log = getFSEditLog(storage);
log.initJournalsForWrite();
assertEquals("[[1,100], [101,200], [201,300], [301,400]]",
log.getEditLogManifest(1).toString());
assertEquals("[[1,100], [101,200], [201,300], [301,400]]" +
" CommittedTxId: 400", log.getEditLogManifest(1).toString());
// Case where one directory has an earlier finalized log, followed
// by a gap. The returned manifest should start after the gap.
@ -1059,7 +1059,7 @@ public class TestEditLog {
"[301,400]|[401,500]");
log = getFSEditLog(storage);
log.initJournalsForWrite();
assertEquals("[[301,400], [401,500]]",
assertEquals("[[301,400], [401,500]] CommittedTxId: 500",
log.getEditLogManifest(1).toString());
// Case where different directories have different length logs
@ -1069,9 +1069,9 @@ public class TestEditLog {
"[1,50]|[101,200]"); // short log at 1
log = getFSEditLog(storage);
log.initJournalsForWrite();
assertEquals("[[1,100], [101,200]]",
assertEquals("[[1,100], [101,200]] CommittedTxId: 200",
log.getEditLogManifest(1).toString());
assertEquals("[[101,200]]",
assertEquals("[[101,200]] CommittedTxId: 200",
log.getEditLogManifest(101).toString());
// Case where the first storage has an inprogress while
@ -1082,9 +1082,9 @@ public class TestEditLog {
"[1,100]|[101,200]");
log = getFSEditLog(storage);
log.initJournalsForWrite();
assertEquals("[[1,100], [101,200]]",
assertEquals("[[1,100], [101,200]] CommittedTxId: 200",
log.getEditLogManifest(1).toString());
assertEquals("[[101,200]]",
assertEquals("[[101,200]] CommittedTxId: 200",
log.getEditLogManifest(101).toString());
}

View File

@ -33,6 +33,7 @@ import java.util.EnumMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import com.google.protobuf.ByteString;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
@ -79,6 +80,23 @@ public class TestEditLogFileInputStream {
elis.close();
}
@Test
public void testByteStringLog() throws Exception {
ByteString bytes = ByteString.copyFrom(FAKE_LOG_DATA);
EditLogInputStream elis = EditLogFileInputStream.fromByteString(bytes,
HdfsServerConstants.INVALID_TXID, HdfsServerConstants.INVALID_TXID,
true);
// Read the edit log and verify that all of the data is present
EnumMap<FSEditLogOpCodes, Holder<Integer>> counts = FSImageTestUtil
.countEditLogOpTypes(elis);
assertThat(counts.get(FSEditLogOpCodes.OP_ADD).held, is(1));
assertThat(counts.get(FSEditLogOpCodes.OP_SET_GENSTAMP_V1).held, is(1));
assertThat(counts.get(FSEditLogOpCodes.OP_CLOSE).held, is(1));
assertEquals(FAKE_LOG_DATA.length, elis.length());
elis.close();
}
/**
* Regression test for HDFS-8965 which verifies that
* FSEditLogFileInputStream#scanOp verifies Op checksums.

View File

@ -399,7 +399,7 @@ public class TestFileJournalManager {
FileJournalManager.matchEditLogs(badDir);
}
private static EditLogInputStream getJournalInputStream(JournalManager jm,
private static EditLogInputStream getJournalInputStream(FileJournalManager jm,
long txId, boolean inProgressOk) throws IOException {
final PriorityQueue<EditLogInputStream> allStreams =
new PriorityQueue<EditLogInputStream>(64,

View File

@ -176,7 +176,7 @@ public class TestGenericJournalConf {
@Override
public void selectInputStreams(Collection<EditLogInputStream> streams,
long fromTxnId, boolean inProgressOk) {
long fromTxnId, boolean inProgressOk, boolean onlyDurableTxns) {
}
@Override

View File

@ -368,11 +368,11 @@ public class TestNNStorageRetentionManager {
public Void answer(InvocationOnMock invocation) throws Throwable {
Object[] args = invocation.getArguments();
journalSet.selectInputStreams((Collection<EditLogInputStream>)args[0],
(Long)args[1], (Boolean)args[2]);
(Long)args[1], (Boolean)args[2], (Boolean)args[3]);
return null;
}
}).when(mockLog).selectInputStreams(Mockito.anyCollection(),
Mockito.anyLong(), Mockito.anyBoolean());
Mockito.anyLong(), Mockito.anyBoolean(), Mockito.anyBoolean());
return mockLog;
}
}

View File

@ -198,7 +198,7 @@ public class TestFailureToReadEdits {
// This op should get applied just fine.
assertTrue(fs.mkdirs(new Path(TEST_DIR2)));
// This is the op the mocking will cause to fail to be read.
assertTrue(fs.mkdirs(new Path(TEST_DIR3)));
@ -220,7 +220,7 @@ public class TestFailureToReadEdits {
// Null because it hasn't been created yet.
assertNull(NameNodeAdapter.getFileInfo(nn1,
TEST_DIR3, false));
// Now let the standby read ALL the edits.
answer.setThrowExceptionOnRead(false);
HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
@ -340,7 +340,8 @@ public class TestFailureToReadEdits {
FSEditLog spyEditLog = NameNodeAdapter.spyOnEditLog(nn1);
LimitedEditLogAnswer answer = new LimitedEditLogAnswer();
doAnswer(answer).when(spyEditLog).selectInputStreams(
anyLong(), anyLong(), (MetaRecoveryContext)anyObject(), anyBoolean());
anyLong(), anyLong(), (MetaRecoveryContext)anyObject(), anyBoolean(),
anyBoolean());
return answer;
}

View File

@ -0,0 +1,339 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.namenode.ha;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import java.io.File;
import java.io.IOException;
import java.net.URI;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
import org.apache.hadoop.hdfs.server.namenode.NNStorage;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import com.google.common.base.Joiner;
import com.google.common.collect.Lists;
/**
* Test cases for in progress tailing edit logs by
* the standby node.
*/
public class TestStandbyInProgressTail {
private static final Log LOG =
LogFactory.getLog(TestStandbyInProgressTail.class);
private Configuration conf;
private MiniQJMHACluster qjmhaCluster;
private MiniDFSCluster cluster;
private NameNode nn0;
private NameNode nn1;
@Before
public void startUp() throws IOException {
conf = new Configuration();
// Set period of tail edits to a large value (20 mins) for test purposes
conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 20 * 60);
conf.setBoolean(DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY, true);
HAUtil.setAllowStandbyReads(conf, true);
qjmhaCluster = new MiniQJMHACluster.Builder(conf).build();
cluster = qjmhaCluster.getDfsCluster();
// Get NameNode from cluster to future manual control
nn0 = cluster.getNameNode(0);
nn1 = cluster.getNameNode(1);
}
@After
public void tearDown() throws IOException {
if (qjmhaCluster != null) {
qjmhaCluster.shutdown();
}
}
@Test
public void testDefault() throws Exception {
if (qjmhaCluster != null) {
qjmhaCluster.shutdown();
}
conf = new Configuration();
// Set period of tail edits to a large value (20 mins) for test purposes
conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 20 * 60);
conf.setBoolean(DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY, false);
HAUtil.setAllowStandbyReads(conf, true);
qjmhaCluster = new MiniQJMHACluster.Builder(conf).build();
cluster = qjmhaCluster.getDfsCluster();
try {
// During HA startup, both nodes should be in
// standby and we shouldn't have any edits files
// in any edits directory!
List<URI> allDirs = Lists.newArrayList();
allDirs.addAll(cluster.getNameDirs(0));
allDirs.addAll(cluster.getNameDirs(1));
assertNoEditFiles(allDirs);
// Set the first NN to active, make sure it creates edits
// in its own dirs and the shared dir. The standby
// should still have no edits!
cluster.transitionToActive(0);
assertEditFiles(cluster.getNameDirs(0),
NNStorage.getInProgressEditsFileName(1));
assertNoEditFiles(cluster.getNameDirs(1));
cluster.getNameNode(0).getRpcServer().mkdirs("/test",
FsPermission.createImmutable((short) 0755), true);
cluster.getNameNode(1).getNamesystem().getEditLogTailer().doTailEdits();
// StandbyNameNode should not finish tailing in-progress logs
assertNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
"/test", true));
// Restarting the standby should not finalize any edits files
// in the shared directory when it starts up!
cluster.restartNameNode(1);
assertEditFiles(cluster.getNameDirs(0),
NNStorage.getInProgressEditsFileName(1));
assertNoEditFiles(cluster.getNameDirs(1));
// Additionally it should not have applied any in-progress logs
// at start-up -- otherwise, it would have read half-way into
// the current log segment, and on the next roll, it would have to
// either replay starting in the middle of the segment (not allowed)
// or double-replay the edits (incorrect).
assertNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
"/test", true));
cluster.getNameNode(0).getRpcServer().mkdirs("/test2",
FsPermission.createImmutable((short) 0755), true);
// If we restart NN0, it'll come back as standby, and we can
// transition NN1 to active and make sure it reads edits correctly.
cluster.restartNameNode(0);
cluster.transitionToActive(1);
// NN1 should have both the edits that came before its restart,
// and the edits that came after its restart.
assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
"/test", true));
assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
"/test2", true));
} finally {
if (qjmhaCluster != null) {
qjmhaCluster.shutdown();
}
}
}
@Test
public void testSetup() throws Exception {
// During HA startup, both nodes should be in
// standby and we shouldn't have any edits files
// in any edits directory!
List<URI> allDirs = Lists.newArrayList();
allDirs.addAll(cluster.getNameDirs(0));
allDirs.addAll(cluster.getNameDirs(1));
assertNoEditFiles(allDirs);
// Set the first NN to active, make sure it creates edits
// in its own dirs and the shared dir. The standby
// should still have no edits!
cluster.transitionToActive(0);
assertEditFiles(cluster.getNameDirs(0),
NNStorage.getInProgressEditsFileName(1));
assertNoEditFiles(cluster.getNameDirs(1));
cluster.getNameNode(0).getRpcServer().mkdirs("/test",
FsPermission.createImmutable((short) 0755), true);
nn1.getNamesystem().getEditLogTailer().doTailEdits();
// After waiting for 5 seconds, StandbyNameNode should finish tailing
// in-progress logs
assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
"/test", true));
// Restarting the standby should not finalize any edits files
// in the shared directory when it starts up!
cluster.restartNameNode(1);
assertEditFiles(cluster.getNameDirs(0),
NNStorage.getInProgressEditsFileName(1));
assertNoEditFiles(cluster.getNameDirs(1));
// Because we're using in-progress tailer, this should not be null
assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
"/test", true));
cluster.getNameNode(0).getRpcServer().mkdirs("/test2",
FsPermission.createImmutable((short) 0755), true);
// If we restart NN0, it'll come back as standby, and we can
// transition NN1 to active and make sure it reads edits correctly.
cluster.restartNameNode(0);
cluster.transitionToActive(1);
// NN1 should have both the edits that came before its restart,
// and the edits that came after its restart.
assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
"/test", true));
assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
"/test2", true));
}
@Test
public void testHalfStartInProgressTail() throws Exception {
// Set the first NN to active, make sure it creates edits
// in its own dirs and the shared dir. The standby
// should still have no edits!
cluster.transitionToActive(0);
assertEditFiles(cluster.getNameDirs(0),
NNStorage.getInProgressEditsFileName(1));
assertNoEditFiles(cluster.getNameDirs(1));
cluster.getNameNode(0).getRpcServer().mkdirs("/test",
FsPermission.createImmutable((short) 0755), true);
nn1.getNamesystem().getEditLogTailer().doTailEdits();
// StandbyNameNode should tail the in-progress edit
assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test", true));
// Create a new edit and finalized it
cluster.getNameNode(0).getRpcServer().mkdirs("/test2",
FsPermission.createImmutable((short) 0755), true);
nn0.getRpcServer().rollEditLog();
// StandbyNameNode shouldn't tail the edit since we do not call the method
assertNull(NameNodeAdapter.getFileInfo(nn1, "/test2", true));
// Create a new in-progress edit and let SBNN do the tail
cluster.getNameNode(0).getRpcServer().mkdirs("/test3",
FsPermission.createImmutable((short) 0755), true);
nn1.getNamesystem().getEditLogTailer().doTailEdits();
// StandbyNameNode should tail the finalized edit and the new in-progress
assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test", true));
assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test2", true));
assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test3", true));
}
@Test
public void testInitStartInProgressTail() throws Exception {
// Set the first NN to active, make sure it creates edits
// in its own dirs and the shared dir. The standby
// should still have no edits!
cluster.transitionToActive(0);
assertEditFiles(cluster.getNameDirs(0),
NNStorage.getInProgressEditsFileName(1));
assertNoEditFiles(cluster.getNameDirs(1));
cluster.getNameNode(0).getRpcServer().mkdirs("/test",
FsPermission.createImmutable((short) 0755), true);
cluster.getNameNode(0).getRpcServer().mkdirs("/test2",
FsPermission.createImmutable((short) 0755), true);
nn0.getRpcServer().rollEditLog();
cluster.getNameNode(0).getRpcServer().mkdirs("/test3",
FsPermission.createImmutable((short) 0755), true);
assertNull(NameNodeAdapter.getFileInfo(nn1, "/test", true));
assertNull(NameNodeAdapter.getFileInfo(nn1, "/test2", true));
assertNull(NameNodeAdapter.getFileInfo(nn1, "/test3", true));
nn1.getNamesystem().getEditLogTailer().doTailEdits();
// StandbyNameNode shoudl tail the finalized edit and the new in-progress
assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test", true));
assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test2", true));
assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test3", true));
}
@Test
public void testNewStartInProgressTail() throws Exception {
cluster.transitionToActive(0);
assertEditFiles(cluster.getNameDirs(0),
NNStorage.getInProgressEditsFileName(1));
assertNoEditFiles(cluster.getNameDirs(1));
cluster.getNameNode(0).getRpcServer().mkdirs("/test",
FsPermission.createImmutable((short) 0755), true);
cluster.getNameNode(0).getRpcServer().mkdirs("/test2",
FsPermission.createImmutable((short) 0755), true);
nn1.getNamesystem().getEditLogTailer().doTailEdits();
nn0.getRpcServer().rollEditLog();
assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test", true));
assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test2", true));
cluster.getNameNode(0).getRpcServer().mkdirs("/test3",
FsPermission.createImmutable((short) 0755), true);
nn1.getNamesystem().getEditLogTailer().doTailEdits();
// StandbyNameNode shoudl tail the finalized edit and the new in-progress
assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test", true));
assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test2", true));
assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test3", true));
}
/**
* Check that no edits files are present in the given storage dirs.
*/
private static void assertNoEditFiles(Iterable<URI> dirs) throws IOException {
assertEditFiles(dirs);
}
/**
* Check that the given list of edits files are present in the given storage
* dirs.
*/
private static void assertEditFiles(Iterable<URI> dirs, String... files)
throws IOException {
for (URI u : dirs) {
File editDirRoot = new File(u.getPath());
File editDir = new File(editDirRoot, "current");
GenericTestUtils.assertExists(editDir);
if (files.length == 0) {
LOG.info("Checking no edit files exist in " + editDir);
} else {
LOG.info("Checking for following edit files in " + editDir
+ ": " + Joiner.on(",").join(files));
}
GenericTestUtils.assertGlobEquals(editDir, "edits_.*", files);
}
}
}