diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java
index a4af34666ef..a539a7a13d5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java
@@ -63,6 +63,15 @@ public final class ExitUtil {
public static ExitException getFirstExitException() {
return firstExitException;
}
+
+ /**
+ * Reset the tracking of process termination. This is for use
+ * in unit tests where one test in the suite expects an exit
+ * but others do not.
+ */
+ public static void resetFirstExitException() {
+ firstExitException = null;
+ }
/**
* Terminate the current process. Note that terminate is the *only* method
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-3077.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-3077.txt
new file mode 100644
index 00000000000..6c1930964c7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-3077.txt
@@ -0,0 +1,4 @@
+Changes for HDFS-3077 branch.
+This will be merged into the main CHANGES.txt when the branch is merged.
+
+HDFS-3077. Quorum-based protocol for reading and writing edit logs. Contributed by Todd Lipcon based on initial work from Brandon Li and Hari Mankude.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
index 3461d6afe4c..e152627bc8f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
@@ -8,6 +8,9 @@
+
+
+
@@ -273,4 +276,11 @@
+
+
+
+
+
+
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index 02d9857fe22..b5ff4845fb7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -177,6 +177,25 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+ journal
+ generate-sources
+
+ compile
+
+
+ false
+ ${project.build.directory}/generated-src/main/jsp
+ ${project.build.directory}/journal-jsp-servlet-definitions.xml
+ org.apache.hadoop.hdfs.server.journalservice
+
+ ${basedir}/src/main/webapps/journal
+
+ *.jsp
+
+
+
+ datanodegenerate-sources
@@ -284,6 +303,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
@@ -299,6 +319,9 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index 06b8b5acf39..fbe0d11ce30 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -30,6 +30,7 @@ function print_usage(){
echo " namenode -format format the DFS filesystem"
echo " secondarynamenode run the DFS secondary namenode"
echo " namenode run the DFS namenode"
+ echo " journalnode run the DFS journalnode"
echo " zkfc run the ZK Failover Controller daemon"
echo " datanode run a DFS datanode"
echo " dfsadmin run a DFS admin client"
@@ -90,6 +91,9 @@ elif [ "$COMMAND" = "datanode" ] ; then
else
HADOOP_OPTS="$HADOOP_OPTS -server $HADOOP_DATANODE_OPTS"
fi
+elif [ "$COMMAND" = "journalnode" ] ; then
+ CLASS='org.apache.hadoop.hdfs.qjournal.server.JournalNode'
+ HADOOP_OPTS="$HADOOP_OPTS $HADOOP_JOURNALNODE_OPTS"
elif [ "$COMMAND" = "dfs" ] ; then
CLASS=org.apache.hadoop.fs.FsShell
HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index ddadbdd44f7..535de776533 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -366,4 +366,37 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final boolean DFS_HA_AUTO_FAILOVER_ENABLED_DEFAULT = false;
public static final String DFS_HA_ZKFC_PORT_KEY = "dfs.ha.zkfc.port";
public static final int DFS_HA_ZKFC_PORT_DEFAULT = 8019;
+
+ // Journal-node related configs. These are read on the JN side.
+ public static final String DFS_JOURNALNODE_EDITS_DIR_KEY = "dfs.journalnode.edits.dir";
+ public static final String DFS_JOURNALNODE_EDITS_DIR_DEFAULT = "/tmp/hadoop/dfs/journalnode/";
+ public static final String DFS_JOURNALNODE_RPC_ADDRESS_KEY = "dfs.journalnode.rpc-address";
+ public static final int DFS_JOURNALNODE_RPC_PORT_DEFAULT = 8485;
+ public static final String DFS_JOURNALNODE_RPC_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_JOURNALNODE_RPC_PORT_DEFAULT;
+
+ public static final String DFS_JOURNALNODE_HTTP_ADDRESS_KEY = "dfs.journalnode.http-address";
+ public static final int DFS_JOURNALNODE_HTTP_PORT_DEFAULT = 8480;
+ public static final String DFS_JOURNALNODE_HTTP_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_JOURNALNODE_HTTP_PORT_DEFAULT;
+
+ public static final String DFS_JOURNALNODE_KEYTAB_FILE_KEY = "dfs.journalnode.keytab.file";
+ public static final String DFS_JOURNALNODE_USER_NAME_KEY = "dfs.journalnode.kerberos.principal";
+ public static final String DFS_JOURNALNODE_INTERNAL_SPNEGO_USER_NAME_KEY = "dfs.journalnode.kerberos.internal.spnego.principal";
+
+ // Journal-node related configs for the client side.
+ public static final String DFS_QJOURNAL_QUEUE_SIZE_LIMIT_KEY = "dfs.qjournal.queued-edits.limit.mb";
+ public static final int DFS_QJOURNAL_QUEUE_SIZE_LIMIT_DEFAULT = 10;
+
+ // Quorum-journal timeouts for various operations. Unlikely to need
+ // to be tweaked, but configurable just in case.
+ public static final String DFS_QJOURNAL_START_SEGMENT_TIMEOUT_KEY = "dfs.qjournal.start-segment.timeout.ms";
+ public static final String DFS_QJOURNAL_PREPARE_RECOVERY_TIMEOUT_KEY = "dfs.qjournal.prepare-recovery.timeout.ms";
+ public static final String DFS_QJOURNAL_ACCEPT_RECOVERY_TIMEOUT_KEY = "dfs.qjournal.accept-recovery.timeout.ms";
+ public static final String DFS_QJOURNAL_FINALIZE_SEGMENT_TIMEOUT_KEY = "dfs.qjournal.finalize-segment.timeout.ms";
+ public static final String DFS_QJOURNAL_SELECT_INPUT_STREAMS_TIMEOUT_KEY = "dfs.qjournal.select-input-streams.timeout.ms";
+ public static final int DFS_QJOURNAL_START_SEGMENT_TIMEOUT_DEFAULT = 20000;
+ public static final int DFS_QJOURNAL_PREPARE_RECOVERY_TIMEOUT_DEFAULT = 20000;
+ public static final int DFS_QJOURNAL_ACCEPT_RECOVERY_TIMEOUT_DEFAULT = 60000;
+ public static final int DFS_QJOURNAL_FINALIZE_SEGMENT_TIMEOUT_DEFAULT = 20000;
+ public static final int DFS_QJOURNAL_SELECT_INPUT_STREAMS_TIMEOUT_DEFAULT = 20000;
}
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 93fe2497e1c..61720a58cde 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -328,12 +328,15 @@ public class PBHelper {
}
public static RemoteEditLogProto convert(RemoteEditLog log) {
- return RemoteEditLogProto.newBuilder().setEndTxId(log.getEndTxId())
- .setStartTxId(log.getStartTxId()).build();
+ return RemoteEditLogProto.newBuilder()
+ .setStartTxId(log.getStartTxId())
+ .setEndTxId(log.getEndTxId())
+ .setIsInProgress(log.isInProgress()).build();
}
public static RemoteEditLog convert(RemoteEditLogProto l) {
- return new RemoteEditLog(l.getStartTxId(), l.getEndTxId());
+ return new RemoteEditLog(l.getStartTxId(), l.getEndTxId(),
+ l.getIsInProgress());
}
public static RemoteEditLogManifestProto convert(
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java
new file mode 100644
index 00000000000..d14fe825f3f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java
@@ -0,0 +1,114 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.client;
+
+import java.net.URL;
+
+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.GetJournalStateResponseProto;
+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;
+import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+/**
+ * Interface for a remote log which is only communicated with asynchronously.
+ * This is essentially a wrapper around {@link QJournalProtocol} with the key
+ * differences being:
+ *
+ *
+ *
All methods return {@link ListenableFuture}s instead of synchronous
+ * objects.
+ *
The {@link RequestInfo} objects are created by the underlying
+ * implementation.
+ *
+ */
+interface AsyncLogger {
+
+ /**
+ * Send a batch of edits to the logger.
+ * @param firstTxnId the first txid of the edits.
+ * @param numTxns the number of transactions in the batch
+ * @param data the actual data to be sent
+ */
+ public ListenableFuture sendEdits(
+ final long firstTxnId, final int numTxns, final byte[] data);
+
+ /**
+ * Begin writing a new log segment.
+ *
+ * @param txid the first txid to be written to the new log
+ */
+ public ListenableFuture startLogSegment(long txid);
+
+ /**
+ * Finalize a log segment.
+ *
+ * @param startTxId the first txid that was written to the segment
+ * @param endTxId the last txid that was written to the segment
+ */
+ public ListenableFuture finalizeLogSegment(
+ long startTxId, long endTxId);
+
+ /**
+ * @return the state of the last epoch on the target node.
+ */
+ public ListenableFuture getJournalState();
+
+ /**
+ * Begin a new epoch on the target node.
+ */
+ public ListenableFuture newEpoch(long epoch);
+
+ /**
+ * Fetch the list of edit logs available on the remote node.
+ */
+ public ListenableFuture getEditLogManifest(
+ long fromTxnId);
+
+ /**
+ * Prepare recovery. See the HDFS-3077 design document for details.
+ */
+ public ListenableFuture prepareRecovery(
+ long segmentTxId);
+
+ /**
+ * Accept a recovery proposal. See the HDFS-3077 design document for details.
+ */
+ public ListenableFuture acceptRecovery(SegmentStateProto log,
+ URL fromUrl);
+
+ /**
+ * Set the epoch number used for all future calls.
+ */
+ public void setEpoch(long e);
+
+ /**
+ * Build an HTTP URL to fetch the log segment with the given startTxId.
+ */
+ public URL buildURLToFetchLogs(long segmentTxId);
+
+ /**
+ * Tear down any resources, connections, etc. The proxy may not be used
+ * after this point, and any in-flight RPCs may throw an exception.
+ */
+ public void close();
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
new file mode 100644
index 00000000000..d67911aaba0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
@@ -0,0 +1,272 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.client;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.List;
+import java.util.Map;
+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.GetEditLogManifestResponseProto;
+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;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ListenableFuture;
+
+/**
+ * Wrapper around a set of Loggers, taking care of fanning out
+ * calls to the underlying loggers and constructing corresponding
+ * {@link QuorumCall} instances.
+ */
+class AsyncLoggerSet {
+ static final Log LOG = LogFactory.getLog(AsyncLoggerSet.class);
+
+ private static final int NEWEPOCH_TIMEOUT_MS = 10000;
+
+ private final List loggers;
+
+ private static final long INVALID_EPOCH = -1;
+ private long myEpoch = INVALID_EPOCH;
+
+ public AsyncLoggerSet(List loggers) {
+ this.loggers = ImmutableList.copyOf(loggers);
+ }
+
+ /**
+ * Fence any previous writers, and obtain a unique epoch number
+ * for write-access to the journal nodes.
+ *
+ * @param nsInfo the expected namespace information. If the remote
+ * node does not match with this namespace, the request will be rejected.
+ * @return the new, unique epoch number
+ * @throws IOException
+ */
+ Map createNewUniqueEpoch(
+ NamespaceInfo nsInfo) throws IOException {
+ Preconditions.checkState(myEpoch == -1,
+ "epoch already created: epoch=" + myEpoch);
+
+ Map lastPromises =
+ waitForWriteQuorum(getJournalState(), NEWEPOCH_TIMEOUT_MS);
+
+ long maxPromised = Long.MIN_VALUE;
+ for (GetJournalStateResponseProto resp : lastPromises.values()) {
+ maxPromised = Math.max(maxPromised, resp.getLastPromisedEpoch());
+ }
+ assert maxPromised >= 0;
+
+ long myEpoch = maxPromised + 1;
+ Map resps =
+ waitForWriteQuorum(newEpoch(nsInfo, myEpoch), NEWEPOCH_TIMEOUT_MS);
+ this.myEpoch = myEpoch;
+ setEpoch(myEpoch);
+ return resps;
+ }
+
+ private void setEpoch(long e) {
+ for (AsyncLogger l : loggers) {
+ l.setEpoch(e);
+ }
+ }
+
+ /**
+ * @return the epoch number for this writer. This may only be called after
+ * a successful call to {@link #createNewUniqueEpoch(NamespaceInfo)}.
+ */
+ long getEpoch() {
+ Preconditions.checkState(myEpoch != INVALID_EPOCH,
+ "No epoch created yet");
+ return myEpoch;
+ }
+
+ /**
+ * Close all of the underlying loggers.
+ */
+ void close() {
+ for (AsyncLogger logger : loggers) {
+ logger.close();
+ }
+ }
+
+
+ /**
+ * Wait for a quorum of loggers to respond to the given call. If a quorum
+ * can't be achieved, throws a QuorumException.
+ * @param q the quorum call
+ * @param timeoutMs the number of millis to wait
+ * @return a map of successful results
+ * @throws QuorumException if a quorum doesn't respond with success
+ * @throws IOException if the thread is interrupted or times out
+ */
+ Map waitForWriteQuorum(QuorumCall q,
+ int timeoutMs) throws IOException {
+ int majority = getMajoritySize();
+ try {
+ q.waitFor(
+ loggers.size(), // either all respond
+ majority, // or we get a majority successes
+ majority, // or we get a majority failures,
+ timeoutMs);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new IOException("Interrupted waiting for quorum results");
+ } catch (TimeoutException e) {
+ throw new IOException("Timed out waiting " + timeoutMs + " for write quorum");
+ }
+
+ if (q.countSuccesses() < majority) {
+ q.rethrowException("Got too many exceptions to achieve quorum size " +
+ getMajorityString());
+ }
+
+ return q.getResults();
+ }
+
+ /**
+ * @return the number of nodes which are required to obtain a quorum.
+ */
+ int getMajoritySize() {
+ return loggers.size() / 2 + 1;
+ }
+
+ /**
+ * @return a textual description of the majority size (eg "2/3" or "3/5")
+ */
+ String getMajorityString() {
+ return getMajoritySize() + "/" + loggers.size();
+ }
+
+ /**
+ * @return the number of loggers behind this set
+ */
+ int size() {
+ return loggers.size();
+ }
+
+ /**
+ * @return the (mutable) list of loggers, for use in tests to
+ * set up spies
+ */
+ @VisibleForTesting
+ List getLoggersForTests() {
+ return loggers;
+ }
+
+ ///////////////////////////////////////////////////////////////////////////
+ // The rest of this file is simply boilerplate wrappers which fan-out the
+ // various IPC calls to the underlying AsyncLoggers and wrap the result
+ // in a QuorumCall.
+ ///////////////////////////////////////////////////////////////////////////
+
+ private QuorumCall getJournalState() {
+ Map> calls =
+ Maps.newHashMap();
+ for (AsyncLogger logger : loggers) {
+ calls.put(logger, logger.getJournalState());
+ }
+ return QuorumCall.create(calls);
+ }
+
+ private QuorumCall newEpoch(
+ NamespaceInfo nsInfo,
+ long epoch) {
+ Map> calls =
+ Maps.newHashMap();
+ for (AsyncLogger logger : loggers) {
+ calls.put(logger, logger.newEpoch(epoch));
+ }
+ return QuorumCall.create(calls);
+ }
+
+ public QuorumCall startLogSegment(
+ long txid) {
+ Map> calls = Maps.newHashMap();
+ for (AsyncLogger logger : loggers) {
+ calls.put(logger, logger.startLogSegment(txid));
+ }
+ return QuorumCall.create(calls);
+ }
+
+ public QuorumCall finalizeLogSegment(long firstTxId,
+ long lastTxId) {
+ Map> calls = Maps.newHashMap();
+ for (AsyncLogger logger : loggers) {
+ calls.put(logger, logger.finalizeLogSegment(firstTxId, lastTxId));
+ }
+ return QuorumCall.create(calls);
+ }
+
+ public QuorumCall sendEdits(
+ long firstTxnId, int numTxns, byte[] data) {
+ Map> calls = Maps.newHashMap();
+ for (AsyncLogger logger : loggers) {
+ ListenableFuture future =
+ logger.sendEdits(firstTxnId, numTxns, data);
+ calls.put(logger, future);
+ }
+ return QuorumCall.create(calls);
+ }
+
+ public QuorumCall
+ getEditLogManifest(long fromTxnId) {
+ Map> calls
+ = Maps.newHashMap();
+ for (AsyncLogger logger : loggers) {
+ ListenableFuture future =
+ logger.getEditLogManifest(fromTxnId);
+ calls.put(logger, future);
+ }
+ return QuorumCall.create(calls);
+ }
+
+ QuorumCall
+ prepareRecovery(long segmentTxId) {
+ Map> calls
+ = Maps.newHashMap();
+ for (AsyncLogger logger : loggers) {
+ ListenableFuture future =
+ logger.prepareRecovery(segmentTxId);
+ calls.put(logger, future);
+ }
+ return QuorumCall.create(calls);
+ }
+
+ QuorumCall
+ acceptRecovery(SegmentStateProto log, URL fromURL) {
+ Map> calls
+ = Maps.newHashMap();
+ for (AsyncLogger logger : loggers) {
+ ListenableFuture future =
+ logger.acceptRecovery(log, fromURL);
+ calls.put(logger, future);
+ }
+ return QuorumCall.create(calls);
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
new file mode 100644
index 00000000000..952f55df6b8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
@@ -0,0 +1,330 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.client;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+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.GetJournalStateResponseProto;
+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;
+import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
+import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolPB;
+import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.qjournal.server.GetJournalEditServlet;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.google.common.util.concurrent.UncaughtExceptionHandlers;
+
+/**
+ * Channel to a remote JournalNode using Hadoop IPC.
+ * All of the calls are run on a separate thread, and return
+ * {@link ListenableFuture} instances to wait for their result.
+ * This allows calls to be bound together using the {@link QuorumCall}
+ * class.
+ */
+@InterfaceAudience.Private
+public class IPCLoggerChannel implements AsyncLogger {
+
+ private final Configuration conf;
+ private final InetSocketAddress addr;
+ private QJournalProtocol proxy;
+
+ private final ListeningExecutorService executor;
+ private long ipcSerial = 0;
+ private long epoch = -1;
+ private final String journalId;
+ private final NamespaceInfo nsInfo;
+ private int httpPort = -1;
+
+ /**
+ * The number of bytes of edits data still in the queue.
+ */
+ private int queuedEditsSizeBytes = 0;
+
+ /**
+ * The maximum number of bytes that can be pending in the queue.
+ * This keeps the writer from hitting OOME if one of the loggers
+ * starts responding really slowly. Eventually, the queue
+ * overflows and it starts to treat the logger as having errored.
+ */
+ private final int queueSizeLimitBytes;
+
+ public IPCLoggerChannel(Configuration conf,
+ NamespaceInfo nsInfo,
+ String journalId,
+ InetSocketAddress addr) {
+ this.conf = conf;
+ this.nsInfo = nsInfo;
+ this.journalId = journalId;
+ this.addr = addr;
+
+ this.queueSizeLimitBytes = 1024 * 1024 * conf.getInt(
+ DFSConfigKeys.DFS_QJOURNAL_QUEUE_SIZE_LIMIT_KEY,
+ DFSConfigKeys.DFS_QJOURNAL_QUEUE_SIZE_LIMIT_DEFAULT);
+
+ executor = MoreExecutors.listeningDecorator(
+ Executors.newSingleThreadExecutor(
+ new ThreadFactoryBuilder()
+ .setDaemon(true)
+ .setNameFormat("Logger channel to " + addr)
+ .setUncaughtExceptionHandler(
+ UncaughtExceptionHandlers.systemExit())
+ .build()));
+ }
+
+ @Override
+ public synchronized void setEpoch(long epoch) {
+ this.epoch = epoch;
+ }
+
+ @Override
+ public void close() {
+ // No more tasks may be submitted after this point.
+ executor.shutdown();
+ if (proxy != null) {
+ // TODO: this can hang for quite some time if the client
+ // is currently in the middle of a call to a downed JN.
+ // We should instead do this asynchronously, and just stop
+ // making any more calls after this point (eg clear the queue)
+ RPC.stopProxy(proxy);
+ }
+ }
+
+ protected QJournalProtocol getProxy() throws IOException {
+ if (proxy != null) return proxy;
+
+ RPC.setProtocolEngine(conf,
+ QJournalProtocolPB.class, ProtobufRpcEngine.class);
+ QJournalProtocolPB pbproxy = RPC.getProxy(
+ QJournalProtocolPB.class,
+ RPC.getProtocolVersion(QJournalProtocolPB.class),
+ addr, conf);
+ proxy = new QJournalProtocolTranslatorPB(pbproxy);
+ return proxy;
+ }
+
+ @Override
+ public URL buildURLToFetchLogs(long segmentTxId) {
+ Preconditions.checkArgument(segmentTxId > 0,
+ "Invalid segment: %s", segmentTxId);
+ Preconditions.checkState(httpPort != -1,
+ "HTTP port not set yet");
+
+ try {
+ String path = GetJournalEditServlet.buildPath(
+ journalId, segmentTxId, nsInfo);
+ return new URL("http", addr.getHostName(), httpPort, path.toString());
+ } catch (MalformedURLException e) {
+ // should never get here.
+ throw new RuntimeException(e);
+ }
+ }
+
+ private synchronized RequestInfo createReqInfo() {
+ Preconditions.checkState(epoch > 0, "bad epoch: " + epoch);
+ return new RequestInfo(journalId, epoch, ipcSerial++);
+ }
+
+ @VisibleForTesting
+ synchronized long getNextIpcSerial() {
+ return ipcSerial;
+ }
+
+ public synchronized int getQueuedEditsSize() {
+ return queuedEditsSizeBytes;
+ }
+
+ @VisibleForTesting
+ void waitForAllPendingCalls() throws InterruptedException {
+ try {
+ executor.submit(new Runnable() {
+ @Override
+ public void run() {
+ }
+ }).get();
+ } catch (ExecutionException e) {
+ // This can't happen!
+ throw new AssertionError(e);
+ }
+ }
+
+ @Override
+ public ListenableFuture getJournalState() {
+ return executor.submit(new Callable() {
+ @Override
+ public GetJournalStateResponseProto call() throws IOException {
+ GetJournalStateResponseProto ret =
+ getProxy().getJournalState(journalId);
+ httpPort = ret.getHttpPort();
+ return ret;
+ }
+ });
+ }
+
+ @Override
+ public ListenableFuture newEpoch(
+ final long epoch) {
+ return executor.submit(new Callable() {
+ @Override
+ public NewEpochResponseProto call() throws IOException {
+ return getProxy().newEpoch(journalId, nsInfo, epoch);
+ }
+ });
+ }
+
+ @Override
+ public ListenableFuture sendEdits(
+ final long firstTxnId, final int numTxns, final byte[] data) {
+ try {
+ reserveQueueSpace(data.length);
+ } catch (LoggerTooFarBehindException e) {
+ return Futures.immediateFailedFuture(e);
+ }
+ ListenableFuture ret = null;
+ try {
+ ret = executor.submit(new Callable() {
+ @Override
+ public Void call() throws IOException {
+ getProxy().journal(createReqInfo(), firstTxnId, numTxns, data);
+ return null;
+ }
+ });
+ } finally {
+ if (ret == null) {
+ // it didn't successfully get submitted,
+ // so adjust the queue size back down.
+ unreserveQueueSpace(data.length);
+ } else {
+ // It was submitted to the queue, so adjust the length
+ // once the call completes, regardless of whether it
+ // succeeds or fails.
+ Futures.addCallback(ret, new FutureCallback() {
+ @Override
+ public void onFailure(Throwable t) {
+ unreserveQueueSpace(data.length);
+ }
+
+ @Override
+ public void onSuccess(Void t) {
+ unreserveQueueSpace(data.length);
+ }
+ });
+ }
+ }
+ return ret;
+ }
+
+ private synchronized void reserveQueueSpace(int size)
+ throws LoggerTooFarBehindException {
+ Preconditions.checkArgument(size >= 0);
+ if (queuedEditsSizeBytes + size > queueSizeLimitBytes &&
+ queuedEditsSizeBytes > 0) {
+ throw new LoggerTooFarBehindException();
+ }
+ queuedEditsSizeBytes += size;
+ }
+
+ private synchronized void unreserveQueueSpace(int size) {
+ Preconditions.checkArgument(size >= 0);
+ queuedEditsSizeBytes -= size;
+ }
+
+ @Override
+ public ListenableFuture startLogSegment(final long txid) {
+ return executor.submit(new Callable() {
+ @Override
+ public Void call() throws IOException {
+ getProxy().startLogSegment(createReqInfo(), txid);
+ return null;
+ }
+ });
+ }
+
+ @Override
+ public ListenableFuture finalizeLogSegment(
+ final long startTxId, final long endTxId) {
+ return executor.submit(new Callable() {
+ @Override
+ public Void call() throws IOException {
+ getProxy().finalizeLogSegment(createReqInfo(), startTxId, endTxId);
+ return null;
+ }
+ });
+ }
+
+ @Override
+ public ListenableFuture getEditLogManifest(
+ final long fromTxnId) {
+ return executor.submit(new Callable() {
+ @Override
+ public GetEditLogManifestResponseProto call() throws IOException {
+ return getProxy().getEditLogManifest(journalId, fromTxnId);
+ }
+ });
+ }
+
+ @Override
+ public ListenableFuture prepareRecovery(
+ final long segmentTxId) {
+ return executor.submit(new Callable() {
+ @Override
+ public PrepareRecoveryResponseProto call() throws IOException {
+ return getProxy().prepareRecovery(createReqInfo(), segmentTxId);
+ }
+ });
+ }
+
+ @Override
+ public ListenableFuture acceptRecovery(
+ final SegmentStateProto log, final URL url) {
+ return executor.submit(new Callable() {
+ @Override
+ public Void call() throws IOException {
+ getProxy().acceptRecovery(createReqInfo(), log, url);
+ return null;
+ }
+ });
+ }
+
+ @Override
+ public String toString() {
+ return "Channel to journal node " + addr;
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/LoggerTooFarBehindException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/LoggerTooFarBehindException.java
new file mode 100644
index 00000000000..0dbf5bfa5bf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/LoggerTooFarBehindException.java
@@ -0,0 +1,25 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.client;
+
+import java.io.IOException;
+
+class LoggerTooFarBehindException extends IOException {
+ private static final long serialVersionUID = 1L;
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java
new file mode 100644
index 00000000000..52208f5eaf4
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java
@@ -0,0 +1,166 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.client;
+
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.util.Time;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.protobuf.Message;
+import com.google.protobuf.TextFormat;
+
+/**
+ * Represents a set of calls for which a quorum of results is needed.
+ * @param a key used to identify each of the outgoing calls
+ * @param the type of the call result
+ */
+class QuorumCall {
+ private final Map successes = Maps.newHashMap();
+ private final Map exceptions = Maps.newHashMap();
+
+ static QuorumCall create(
+ Map> calls) {
+ final QuorumCall qr = new QuorumCall();
+ for (final Entry> e : calls.entrySet()) {
+ Preconditions.checkArgument(e.getValue() != null,
+ "null future for key: " + e.getKey());
+ Futures.addCallback(e.getValue(), new FutureCallback() {
+ @Override
+ public void onFailure(Throwable t) {
+ qr.addException(e.getKey(), t);
+ }
+
+ @Override
+ public void onSuccess(RESULT res) {
+ qr.addResult(e.getKey(), res);
+ }
+ });
+ }
+ return qr;
+ }
+
+ private QuorumCall() {
+ // Only instantiated from factory method above
+ }
+
+ /**
+ * Wait for the quorum to achieve a certain number of responses.
+ *
+ * Note that, even after this returns, more responses may arrive,
+ * causing the return value of other methods in this class to change.
+ *
+ * @param minResponses return as soon as this many responses have been
+ * received, regardless of whether they are successes or exceptions
+ * @param minSuccesses return as soon as this many successful (non-exception)
+ * responses have been received
+ * @param maxExceptions return as soon as this many exception responses
+ * have been received. Pass 0 to return immediately if any exception is
+ * received.
+ * @param millis the number of milliseconds to wait for
+ * @throws InterruptedException if the thread is interrupted while waiting
+ * @throws TimeoutException if the specified timeout elapses before
+ * achieving the desired conditions
+ */
+ public synchronized void waitFor(
+ int minResponses, int minSuccesses, int maxExceptions,
+ int millis)
+ throws InterruptedException, TimeoutException {
+ long et = Time.monotonicNow() + millis;
+ while (true) {
+ if (minResponses > 0 && countResponses() >= minResponses) return;
+ if (minSuccesses > 0 && countSuccesses() >= minSuccesses) return;
+ if (maxExceptions >= 0 && countExceptions() > maxExceptions) return;
+ long rem = et - Time.monotonicNow();
+ if (rem <= 0) {
+ throw new TimeoutException();
+ }
+ wait(rem);
+ }
+ }
+
+ private synchronized void addResult(KEY k, RESULT res) {
+ successes.put(k, res);
+ notifyAll();
+ }
+
+ private synchronized void addException(KEY k, Throwable t) {
+ exceptions.put(k, t);
+ notifyAll();
+ }
+
+ /**
+ * @return the total number of calls for which a response has been received,
+ * regardless of whether it threw an exception or returned a successful
+ * result.
+ */
+ public synchronized int countResponses() {
+ return successes.size() + exceptions.size();
+ }
+
+ /**
+ * @return the number of calls for which a non-exception response has been
+ * received.
+ */
+ public synchronized int countSuccesses() {
+ return successes.size();
+ }
+
+ /**
+ * @return the number of calls for which an exception response has been
+ * received.
+ */
+ public synchronized int countExceptions() {
+ return exceptions.size();
+ }
+
+ /**
+ * @return the map of successful responses. A copy is made such that this
+ * map will not be further mutated, even if further results arrive for the
+ * quorum.
+ */
+ public synchronized Map getResults() {
+ return Maps.newHashMap(successes);
+ }
+
+ public synchronized void rethrowException(String msg) throws QuorumException {
+ Preconditions.checkState(!exceptions.isEmpty());
+ throw QuorumException.create(msg, successes, exceptions);
+ }
+
+ public static String mapToString(
+ Map map) {
+ StringBuilder sb = new StringBuilder();
+ boolean first = true;
+ for (Map.Entry e : map.entrySet()) {
+ if (!first) {
+ sb.append("\n");
+ }
+ first = false;
+ sb.append(e.getKey()).append(": ")
+ .append(TextFormat.shortDebugString(e.getValue()));
+ }
+ return sb.toString();
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumException.java
new file mode 100644
index 00000000000..f9a6efa38e4
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumException.java
@@ -0,0 +1,85 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.client;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+
+/**
+ * Exception thrown when too many exceptions occur while gathering
+ * responses to a quorum call.
+ */
+class QuorumException extends IOException {
+
+ /**
+ * Create a QuorumException instance with a descriptive message detailing
+ * the underlying exceptions, as well as any successful responses which
+ * were returned.
+ * @param the keys for the quorum calls
+ * @param the success response type
+ * @param successes any successful responses returned
+ * @param exceptions the exceptions returned
+ */
+ public static QuorumException create(
+ String simpleMsg,
+ Map successes,
+ Map exceptions) {
+ Preconditions.checkArgument(!exceptions.isEmpty(),
+ "Must pass exceptions");
+
+ StringBuilder msg = new StringBuilder();
+ msg.append(simpleMsg).append(". ");
+ if (!successes.isEmpty()) {
+ msg.append(successes.size()).append(" successful responses:\n");
+ Joiner.on("\n")
+ .useForNull("null")
+ .withKeyValueSeparator(": ")
+ .appendTo(msg, successes);
+ msg.append("\n");
+ }
+ msg.append(exceptions.size() + " exceptions thrown:\n");
+ boolean isFirst = true;
+
+ for (Map.Entry e : exceptions.entrySet()) {
+ if (!isFirst) {
+ msg.append("\n");
+ }
+ isFirst = false;
+
+ msg.append(e.getKey()).append(": ");
+
+ if (e.getValue() instanceof RuntimeException) {
+ msg.append(StringUtils.stringifyException(e.getValue()));
+ } else {
+ msg.append(e.getValue().getLocalizedMessage());
+ }
+ }
+ return new QuorumException(msg.toString());
+ }
+
+ private QuorumException(String msg) {
+ super(msg);
+ }
+
+ private static final long serialVersionUID = 1L;
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
new file mode 100644
index 00000000000..dc8d4f0f8da
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
@@ -0,0 +1,382 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.client;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URL;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.PriorityQueue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
+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;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
+import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
+import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
+import org.apache.hadoop.hdfs.server.namenode.JournalManager;
+import org.apache.hadoop.hdfs.server.namenode.JournalSet;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ComparisonChain;
+import com.google.common.collect.Lists;
+
+/**
+ * A JournalManager that writes to a set of remote JournalNodes,
+ * requiring a quorum of nodes to ack each write.
+ */
+@InterfaceAudience.Private
+public class QuorumJournalManager implements JournalManager {
+ static final Log LOG = LogFactory.getLog(QuorumJournalManager.class);
+
+ // Timeouts for which the QJM will wait for each of the following actions.
+ private final int startSegmentTimeoutMs;
+ private final int prepareRecoveryTimeoutMs;
+ private final int acceptRecoveryTimeoutMs;
+ private final int finalizeSegmentTimeoutMs;
+ private final int selectInputStreamsTimeoutMs;
+
+ private final Configuration conf;
+ private final URI uri;
+ private final NamespaceInfo nsInfo;
+ private boolean isActiveWriter;
+
+ private final AsyncLoggerSet loggers;
+
+ public QuorumJournalManager(Configuration conf,
+ URI uri, NamespaceInfo nsInfo) throws IOException {
+ Preconditions.checkArgument(conf != null, "must be configured");
+
+ this.conf = conf;
+ this.uri = uri;
+ this.nsInfo = nsInfo;
+ this.loggers = new AsyncLoggerSet(createLoggers());
+
+ // Configure timeouts.
+ this.startSegmentTimeoutMs = conf.getInt(
+ DFSConfigKeys.DFS_QJOURNAL_START_SEGMENT_TIMEOUT_KEY,
+ DFSConfigKeys.DFS_QJOURNAL_START_SEGMENT_TIMEOUT_DEFAULT);
+ this.prepareRecoveryTimeoutMs = conf.getInt(
+ DFSConfigKeys.DFS_QJOURNAL_PREPARE_RECOVERY_TIMEOUT_KEY,
+ DFSConfigKeys.DFS_QJOURNAL_PREPARE_RECOVERY_TIMEOUT_DEFAULT);
+ this.acceptRecoveryTimeoutMs = conf.getInt(
+ DFSConfigKeys.DFS_QJOURNAL_ACCEPT_RECOVERY_TIMEOUT_KEY,
+ DFSConfigKeys.DFS_QJOURNAL_ACCEPT_RECOVERY_TIMEOUT_DEFAULT);
+ this.finalizeSegmentTimeoutMs = conf.getInt(
+ DFSConfigKeys.DFS_QJOURNAL_FINALIZE_SEGMENT_TIMEOUT_KEY,
+ DFSConfigKeys.DFS_QJOURNAL_FINALIZE_SEGMENT_TIMEOUT_DEFAULT);
+ this.selectInputStreamsTimeoutMs = conf.getInt(
+ DFSConfigKeys.DFS_QJOURNAL_SELECT_INPUT_STREAMS_TIMEOUT_KEY,
+ DFSConfigKeys.DFS_QJOURNAL_SELECT_INPUT_STREAMS_TIMEOUT_DEFAULT);
+
+ }
+
+ static String parseJournalId(URI uri) {
+ String path = uri.getPath();
+ Preconditions.checkArgument(path != null && !path.isEmpty(),
+ "Bad URI '%s': must identify journal in path component",
+ uri);
+ String journalId = path.substring(1);
+ checkJournalId(journalId);
+ return journalId;
+ }
+
+ public static void checkJournalId(String jid) {
+ Preconditions.checkArgument(jid != null &&
+ !jid.isEmpty() &&
+ !jid.contains("/") &&
+ !jid.startsWith("."),
+ "bad journal id: " + jid);
+ }
+
+ /**
+ * Run recovery/synchronization for a specific segment.
+ * Postconditions:
+ *
+ *
This segment will be finalized on a majority
+ * of nodes.
+ *
All nodes which contain the finalized segment will
+ * agree on the length.
+ *
+ *
+ * @param segmentTxId the starting txid of the segment
+ * @throws IOException
+ */
+ private void recoverUnclosedSegment(long segmentTxId) throws IOException {
+ Preconditions.checkArgument(segmentTxId > 0);
+ LOG.info("Beginning recovery of unclosed segment starting at txid " +
+ segmentTxId);
+
+ // Step 1. Prepare recovery
+ QuorumCall prepare =
+ loggers.prepareRecovery(segmentTxId);
+ Map prepareResponses=
+ loggers.waitForWriteQuorum(prepare, prepareRecoveryTimeoutMs);
+ LOG.info("Recovery prepare phase complete. Responses: " +
+ QuorumCall.mapToString(prepareResponses));
+
+ // Determine the logger who either:
+ // a) Has already accepted a previous proposal that's higher than any
+ // other
+ //
+ // OR, if no such logger exists:
+ //
+ // b) Has the longest log starting at this transaction ID
+
+ // TODO: we should collect any "ties" and pass the URL for all of them
+ // when syncing, so we can tolerate failure during recovery better.
+ Entry bestEntry = Collections.max(
+ prepareResponses.entrySet(), RECOVERY_COMPARATOR);
+ AsyncLogger bestLogger = bestEntry.getKey();
+ PrepareRecoveryResponseProto bestResponse = bestEntry.getValue();
+
+ // Log the above decision, check invariants.
+ if (bestResponse.hasAcceptedInEpoch()) {
+ LOG.info("Using already-accepted recovery for segment " +
+ "starting at txid " + segmentTxId + ": " +
+ bestEntry);
+ } else if (bestResponse.hasSegmentState()) {
+ LOG.info("Using longest log: " + bestEntry);
+ } else {
+ // TODO: can we get here? what about the following case:
+ // - 3 JNs, JN1, JN2, JN3
+ // - writer starts segment 101 on JN1, then crashes
+ // - during newEpoch(), we saw the segment on JN1 and decide to recover segment 101
+ // - during prepare(), JN1 has actually crashed, and we only talk to JN2 and JN3,
+ // neither of which has any entry for this log.
+ // Write a test case.
+ throw new AssertionError("None of the responses " +
+ "had a log to recover: " + QuorumCall.mapToString(prepareResponses));
+ }
+
+
+ // TODO: check that md5s match up between any "tied" logs
+
+ SegmentStateProto logToSync = bestResponse.getSegmentState();
+ assert segmentTxId == logToSync.getStartTxId();
+
+ URL syncFromUrl = bestLogger.buildURLToFetchLogs(segmentTxId);
+
+ QuorumCall accept = loggers.acceptRecovery(logToSync, syncFromUrl);
+ loggers.waitForWriteQuorum(accept, acceptRecoveryTimeoutMs);
+
+ // TODO:
+ // we should only try to finalize loggers who successfully synced above
+ // eg if a logger was down, we don't want to send the finalize request.
+ // write a test for this!
+
+ QuorumCall finalize =
+ loggers.finalizeLogSegment(logToSync.getStartTxId(), logToSync.getEndTxId());
+ loggers.waitForWriteQuorum(finalize, finalizeSegmentTimeoutMs);
+ }
+
+ private static final Comparator> RECOVERY_COMPARATOR =
+ new Comparator>() {
+ @Override
+ public int compare(
+ Entry a,
+ Entry b) {
+
+ PrepareRecoveryResponseProto r1 = a.getValue();
+ PrepareRecoveryResponseProto r2 = b.getValue();
+
+ if (r1.hasSegmentState() && r2.hasSegmentState()) {
+ assert r1.getSegmentState().getStartTxId() ==
+ r2.getSegmentState().getStartTxId() : "bad args: " + r1 + ", " + r2;
+ }
+
+ return ComparisonChain.start()
+ // If one of them has accepted something and the other hasn't,
+ // use the one with an accepted recovery
+ .compare(r1.hasAcceptedInEpoch(), r2.hasAcceptedInEpoch())
+ // If they both accepted, use the one that's more recent
+ .compare(r1.getAcceptedInEpoch(),
+ r2.getAcceptedInEpoch())
+ // Otherwise, choose based on which log is longer
+ .compare(r1.hasSegmentState(), r2.hasSegmentState())
+ .compare(r1.getSegmentState().getEndTxId(), r2.getSegmentState().getEndTxId())
+ .result();
+ }
+ };
+
+ protected List createLoggers() throws IOException {
+ return createLoggers(conf, uri, nsInfo);
+ }
+
+ static List createLoggers(Configuration conf,
+ URI uri, NamespaceInfo nsInfo) throws IOException {
+ List ret = Lists.newArrayList();
+ List addrs = getLoggerAddresses(uri);
+ String jid = parseJournalId(uri);
+ for (InetSocketAddress addr : addrs) {
+ ret.add(new IPCLoggerChannel(conf, nsInfo, jid, addr));
+ }
+ return ret;
+ }
+
+ private static List getLoggerAddresses(URI uri)
+ throws IOException {
+ String authority = uri.getAuthority();
+ Preconditions.checkArgument(authority != null && !authority.isEmpty(),
+ "URI has no authority: " + uri);
+
+ String[] parts = StringUtils.split(authority, ';');
+ for (int i = 0; i < parts.length; i++) {
+ parts[i] = parts[i].trim();
+ }
+
+ if (parts.length % 2 == 0) {
+ LOG.warn("Quorum journal URI '" + uri + "' has an even number " +
+ "of Journal Nodes specified. This is not recommended!");
+ }
+
+ List addrs = Lists.newArrayList();
+ for (String addr : parts) {
+ addrs.add(NetUtils.createSocketAddr(
+ addr, DFSConfigKeys.DFS_JOURNALNODE_RPC_PORT_DEFAULT));
+ }
+ return addrs;
+ }
+
+ @Override
+ public EditLogOutputStream startLogSegment(long txId) throws IOException {
+ Preconditions.checkState(isActiveWriter,
+ "must recover segments before starting a new one");
+ QuorumCall q = loggers.startLogSegment(txId);
+ loggers.waitForWriteQuorum(q, startSegmentTimeoutMs);
+ return new QuorumOutputStream(loggers);
+ }
+
+ @Override
+ public void finalizeLogSegment(long firstTxId, long lastTxId)
+ throws IOException {
+ QuorumCall q = loggers.finalizeLogSegment(
+ firstTxId, lastTxId);
+ loggers.waitForWriteQuorum(q, finalizeSegmentTimeoutMs);
+ }
+
+ @Override
+ public void setOutputBufferCapacity(int size) {
+ // TODO Auto-generated method stub
+
+ }
+
+ @Override
+ public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException {
+ // TODO Auto-generated method stub
+ }
+
+ @Override
+ public void recoverUnfinalizedSegments() throws IOException {
+ Preconditions.checkState(!isActiveWriter, "already active writer");
+
+ Map resps =
+ loggers.createNewUniqueEpoch(nsInfo);
+ LOG.info("newEpoch(" + loggers.getEpoch() + ") responses:\n" +
+ QuorumCall.mapToString(resps));
+
+ long mostRecentSegmentTxId = Long.MIN_VALUE;
+ for (NewEpochResponseProto r : resps.values()) {
+ if (r.hasLastSegmentTxId()) {
+ mostRecentSegmentTxId = Math.max(mostRecentSegmentTxId,
+ r.getLastSegmentTxId());
+ }
+ }
+
+ // On a completely fresh system, none of the journals have any
+ // segments, so there's nothing to recover.
+ if (mostRecentSegmentTxId != Long.MIN_VALUE) {
+ recoverUnclosedSegment(mostRecentSegmentTxId);
+ }
+ isActiveWriter = true;
+ }
+
+ @Override
+ public void close() throws IOException {
+ loggers.close();
+ }
+
+ @Override
+ public void selectInputStreams(Collection streams,
+ long fromTxnId, boolean inProgressOk) {
+
+ QuorumCall q =
+ loggers.getEditLogManifest(fromTxnId);
+ Map resps;
+ try {
+ resps = loggers.waitForWriteQuorum(q, selectInputStreamsTimeoutMs);
+ } catch (IOException ioe) {
+ // TODO: can we do better here?
+ throw new RuntimeException(ioe);
+ }
+
+ LOG.info("selectInputStream manifests:\n" +
+ QuorumCall.mapToString(resps));
+
+ final PriorityQueue allStreams =
+ new PriorityQueue(64,
+ JournalSet.EDIT_LOG_INPUT_STREAM_COMPARATOR);
+ for (Map.Entry e : resps.entrySet()) {
+ AsyncLogger logger = e.getKey();
+ GetEditLogManifestResponseProto response = e.getValue();
+ RemoteEditLogManifest manifest = PBHelper.convert(response.getManifest());
+
+ for (RemoteEditLog remoteLog : manifest.getLogs()) {
+ URL url = logger.buildURLToFetchLogs(remoteLog.getStartTxId());
+ LOG.info("URL: " + url);
+
+ EditLogInputStream elis = EditLogFileInputStream.fromUrl(
+ url, remoteLog.getStartTxId(), remoteLog.getEndTxId(),
+ remoteLog.isInProgress());
+ allStreams.add(elis);
+ }
+ }
+ JournalSet.chainAndMakeRedundantStreams(
+ streams, allStreams, fromTxnId, inProgressOk);
+ }
+
+ @Override
+ public String toString() {
+ return "Quorum journal manager " + uri;
+ }
+
+ @VisibleForTesting
+ AsyncLoggerSet getLoggerSetForTests() {
+ return loggers;
+ }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumOutputStream.java
new file mode 100644
index 00000000000..bb18f34572a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumOutputStream.java
@@ -0,0 +1,103 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.client;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
+import org.apache.hadoop.hdfs.server.namenode.EditsDoubleBuffer;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
+import org.apache.hadoop.io.DataOutputBuffer;
+
+/**
+ * EditLogOutputStream implementation that writes to a quorum of
+ * remote journals.
+ */
+class QuorumOutputStream extends EditLogOutputStream {
+ private final AsyncLoggerSet loggers;
+ private EditsDoubleBuffer buf;
+
+ public QuorumOutputStream(AsyncLoggerSet loggers) throws IOException {
+ super();
+ this.buf = new EditsDoubleBuffer(256*1024); // TODO: conf
+ this.loggers = loggers;
+ }
+
+ @Override
+ public void write(FSEditLogOp op) throws IOException {
+ buf.writeOp(op);
+ }
+
+ @Override
+ public void writeRaw(byte[] bytes, int offset, int length) throws IOException {
+ buf.writeRaw(bytes, offset, length);
+ }
+
+ @Override
+ public void create() throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (buf != null) {
+ buf.close();
+ buf = null;
+ }
+ }
+
+ @Override
+ public void abort() throws IOException {
+ QuorumJournalManager.LOG.warn("Aborting " + this);
+ buf = null;
+ close();
+ }
+
+ @Override
+ public void setReadyToFlush() throws IOException {
+ buf.setReadyToFlush();
+ }
+
+ @Override
+ protected void flushAndSync() throws IOException {
+ int numReadyBytes = buf.countReadyBytes();
+ if (numReadyBytes > 0) {
+ int numReadyTxns = buf.countReadyTxns();
+ long firstTxToFlush = buf.getFirstReadyTxId();
+
+ assert numReadyTxns > 0;
+
+ // Copy from our double-buffer into a new byte array. This is for
+ // two reasons:
+ // 1) The IPC code has no way of specifying to send only a slice of
+ // a larger array.
+ // 2) because the calls to the underlying nodes are asynchronous, we
+ // need a defensive copy to avoid accidentally mutating the buffer
+ // before it is sent.
+ DataOutputBuffer bufToSend = new DataOutputBuffer(numReadyBytes);
+ buf.flushTo(bufToSend);
+ assert bufToSend.getLength() == numReadyBytes;
+ byte[] data = bufToSend.getData();
+ assert data.length == bufToSend.getLength();
+
+ QuorumCall qcall = loggers.sendEdits(
+ firstTxToFlush, numReadyTxns, data);
+ loggers.waitForWriteQuorum(qcall, 20000); // TODO: configurable timeout
+ }
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java
new file mode 100644
index 00000000000..900eb3635ce
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java
@@ -0,0 +1,114 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.protocol;
+
+import java.io.IOException;
+import java.net.URL;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
+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;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
+import org.apache.hadoop.hdfs.qjournal.server.JournalNode;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.security.KerberosInfo;
+
+/**
+ * Protocol used to communicate between {@link QuorumJournalManager}
+ * and each {@link JournalNode}.
+ *
+ * This is responsible for sending edits as well as coordinating
+ * recovery of the nodes.
+ */
+@KerberosInfo(
+ serverPrincipal = DFSConfigKeys.DFS_JOURNALNODE_USER_NAME_KEY,
+ clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
+@InterfaceAudience.Private
+public interface QJournalProtocol {
+ public static final long versionID = 1L;
+
+ /**
+ * Get the current state of the journal, including the most recent
+ * epoch number and the HTTP port.
+ */
+ public GetJournalStateResponseProto getJournalState(String journalId)
+ throws IOException;
+
+ /**
+ * Begin a new epoch. See the HDFS-3077 design doc for details.
+ */
+ public NewEpochResponseProto newEpoch(String journalId,
+ NamespaceInfo nsInfo, long epoch) throws IOException;
+
+ /**
+ * Journal edit records.
+ * This message is sent by the active name-node to the JournalNodes
+ * to write edits to their local logs.
+ */
+ public void journal(RequestInfo reqInfo,
+ long firstTxnId,
+ int numTxns,
+ byte[] records) throws IOException;
+
+ /**
+ * Start writing to a new log segment on the JournalNode.
+ * Before calling this, one should finalize the previous segment
+ * using {@link #finalizeLogSegment(RequestInfo, long, long)}.
+ *
+ * @param txid the first txid in the new log
+ */
+ public void startLogSegment(RequestInfo reqInfo,
+ long txid) throws IOException;
+
+ /**
+ * Finalize the given log segment on the JournalNode. The segment
+ * is expected to be in-progress and starting at the given startTxId.
+ *
+ * @param startTxId the starting transaction ID of teh log
+ * @param endTxId the expected last transaction in the given log
+ * @throws IOException if no such segment exists
+ */
+ public void finalizeLogSegment(RequestInfo reqInfo,
+ long startTxId, long endTxId) throws IOException;
+
+ /**
+ * @param jid the journal from which to enumerate edits
+ * @param sinceTxId the first transaction which the client cares about
+ * @return a list of edit log segments since the given transaction ID.
+ */
+ public GetEditLogManifestResponseProto getEditLogManifest(
+ String jid, long sinceTxId) throws IOException;
+
+ /**
+ * Begin the recovery process for a given segment. See the HDFS-3077
+ * design document for details.
+ */
+ public PrepareRecoveryResponseProto prepareRecovery(RequestInfo reqInfo,
+ long segmentTxId) throws IOException;
+
+ /**
+ * Accept a proposed recovery for the given transaction ID.
+ */
+ public void acceptRecovery(RequestInfo reqInfo,
+ SegmentStateProto stateToAccept, URL fromUrl) throws IOException;
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/RequestInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/RequestInfo.java
new file mode 100644
index 00000000000..86574f7efdb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/RequestInfo.java
@@ -0,0 +1,54 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class RequestInfo {
+ private String jid;
+ private long epoch;
+ private long ipcSerialNumber;
+
+ public RequestInfo(String jid, long epoch, long ipcSerialNumber) {
+ this.jid = jid;
+ this.epoch = epoch;
+ this.ipcSerialNumber = ipcSerialNumber;
+ }
+
+ public long getEpoch() {
+ return epoch;
+ }
+
+ public void setEpoch(long epoch) {
+ this.epoch = epoch;
+ }
+
+ public String getJournalId() {
+ return jid;
+ }
+
+ public long getIpcSerialNumber() {
+ return ipcSerialNumber;
+ }
+
+ public void setIpcSerialNumber(long ipcSerialNumber) {
+ this.ipcSerialNumber = ipcSerialNumber;
+ }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolPB.java
new file mode 100644
index 00000000000..240bbfbc88f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolPB.java
@@ -0,0 +1,41 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.QJournalProtocolService;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.security.KerberosInfo;
+
+/**
+ * Protocol used to journal edits to a JournalNode participating
+ * in the quorum journal.
+ * Note: This extends the protocolbuffer service based interface to
+ * add annotations required for security.
+ */
+@KerberosInfo(
+ serverPrincipal = DFSConfigKeys.DFS_JOURNALNODE_USER_NAME_KEY,
+ clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
+@ProtocolInfo(protocolName =
+ "org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol",
+ protocolVersion = 1)
+@InterfaceAudience.Private
+public interface QJournalProtocolPB extends
+ QJournalProtocolService.BlockingInterface {
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
new file mode 100644
index 00000000000..4859152fb19
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
@@ -0,0 +1,177 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FinalizeLogSegmentRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FinalizeLogSegmentResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.JournalIdProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.JournalRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.JournalResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+import java.io.IOException;
+import java.net.URL;
+
+/**
+ * Implementation for protobuf service that forwards requests
+ * received on {@link JournalProtocolPB} to the
+ * {@link JournalProtocol} server implementation.
+ */
+@InterfaceAudience.Private
+public class QJournalProtocolServerSideTranslatorPB implements QJournalProtocolPB {
+ /** Server side implementation to delegate the requests to */
+ private final QJournalProtocol impl;
+
+ public QJournalProtocolServerSideTranslatorPB(QJournalProtocol impl) {
+ this.impl = impl;
+ }
+
+ @Override
+ public GetJournalStateResponseProto getJournalState(RpcController controller,
+ GetJournalStateRequestProto request) throws ServiceException {
+ try {
+ return impl.getJournalState(
+ convert(request.getJid()));
+ } catch (IOException ioe) {
+ throw new ServiceException(ioe);
+ }
+ }
+
+ private String convert(JournalIdProto jid) {
+ return jid.getIdentifier();
+ }
+
+ @Override
+ public NewEpochResponseProto newEpoch(RpcController controller,
+ NewEpochRequestProto request) throws ServiceException {
+ try {
+ return impl.newEpoch(
+ request.getJid().getIdentifier(),
+ PBHelper.convert(request.getNsInfo()),
+ request.getEpoch());
+ } catch (IOException ioe) {
+ throw new ServiceException(ioe);
+ }
+ }
+
+ /** @see JournalProtocol#journal */
+ @Override
+ public JournalResponseProto journal(RpcController unused,
+ JournalRequestProto req) throws ServiceException {
+ try {
+ impl.journal(convert(req.getReqInfo()),
+ req.getFirstTxnId(), req.getNumTxns(), req.getRecords()
+ .toByteArray());
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ return JournalResponseProto.newBuilder().build();
+ }
+
+ /** @see JournalProtocol#startLogSegment */
+ @Override
+ public StartLogSegmentResponseProto startLogSegment(RpcController controller,
+ StartLogSegmentRequestProto req) throws ServiceException {
+ try {
+ impl.startLogSegment(convert(req.getReqInfo()),
+ req.getTxid());
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ return StartLogSegmentResponseProto.newBuilder().build();
+ }
+
+ @Override
+ public FinalizeLogSegmentResponseProto finalizeLogSegment(
+ RpcController controller, FinalizeLogSegmentRequestProto req)
+ throws ServiceException {
+ try {
+ impl.finalizeLogSegment(convert(req.getReqInfo()),
+ req.getStartTxId(), req.getEndTxId());
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ return FinalizeLogSegmentResponseProto.newBuilder().build();
+ }
+
+ @Override
+ public GetEditLogManifestResponseProto getEditLogManifest(
+ RpcController controller, GetEditLogManifestRequestProto request)
+ throws ServiceException {
+ try {
+ return impl.getEditLogManifest(
+ request.getJid().getIdentifier(),
+ request.getSinceTxId());
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+
+ @Override
+ public PrepareRecoveryResponseProto prepareRecovery(RpcController controller,
+ PrepareRecoveryRequestProto request) throws ServiceException {
+ try {
+ return impl.prepareRecovery(convert(request.getReqInfo()),
+ request.getSegmentTxId());
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+ @Override
+ public AcceptRecoveryResponseProto acceptRecovery(RpcController controller,
+ AcceptRecoveryRequestProto request) throws ServiceException {
+ try {
+ impl.acceptRecovery(convert(request.getReqInfo()),
+ request.getStateToAccept(),
+ new URL(request.getFromURL()));
+ return AcceptRecoveryResponseProto.getDefaultInstance();
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ }
+
+
+ private RequestInfo convert(
+ QJournalProtocolProtos.RequestInfoProto reqInfo) {
+ return new RequestInfo(
+ reqInfo.getJournalId().getIdentifier(),
+ reqInfo.getEpoch(),
+ reqInfo.getIpcSerialNumber());
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
new file mode 100644
index 00000000000..25b10966a1d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
@@ -0,0 +1,217 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.protocolPB;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.URL;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FinalizeLogSegmentRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.JournalIdProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.JournalRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.AcceptRecoveryRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentRequestProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
+import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+
+import com.google.protobuf.ByteString;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+/**
+ * This class is the client side translator to translate the requests made on
+ * {@link JournalProtocol} interfaces to the RPC server implementing
+ * {@link JournalProtocolPB}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class QJournalProtocolTranslatorPB implements ProtocolMetaInterface,
+ QJournalProtocol, Closeable {
+ /** RpcController is not used and hence is set to null */
+ private final static RpcController NULL_CONTROLLER = null;
+ private final QJournalProtocolPB rpcProxy;
+
+ public QJournalProtocolTranslatorPB(QJournalProtocolPB rpcProxy) {
+ this.rpcProxy = rpcProxy;
+ }
+
+ @Override
+ public void close() {
+ RPC.stopProxy(rpcProxy);
+ }
+
+
+ @Override
+ public GetJournalStateResponseProto getJournalState(String jid)
+ throws IOException {
+ try {
+ GetJournalStateRequestProto req = GetJournalStateRequestProto.newBuilder()
+ .setJid(convertJournalId(jid))
+ .build();
+ return rpcProxy.getJournalState(NULL_CONTROLLER, req);
+ } catch (ServiceException e) {
+ throw ProtobufHelper.getRemoteException(e);
+ }
+ }
+
+ private JournalIdProto convertJournalId(String jid) {
+ return JournalIdProto.newBuilder()
+ .setIdentifier(jid)
+ .build();
+ }
+
+ @Override
+ public NewEpochResponseProto newEpoch(String jid, NamespaceInfo nsInfo,
+ long epoch) throws IOException {
+ try {
+ NewEpochRequestProto req = NewEpochRequestProto.newBuilder()
+ .setJid(convertJournalId(jid))
+ .setNsInfo(PBHelper.convert(nsInfo))
+ .setEpoch(epoch)
+ .build();
+ return rpcProxy.newEpoch(NULL_CONTROLLER, req);
+ } catch (ServiceException e) {
+ throw ProtobufHelper.getRemoteException(e);
+ }
+ }
+
+ @Override
+ public void journal(RequestInfo reqInfo, long firstTxnId, int numTxns,
+ byte[] records) throws IOException {
+ JournalRequestProto req = JournalRequestProto.newBuilder()
+ .setReqInfo(convert(reqInfo))
+ .setFirstTxnId(firstTxnId)
+ .setNumTxns(numTxns)
+ .setRecords(PBHelper.getByteString(records))
+ .build();
+ try {
+ rpcProxy.journal(NULL_CONTROLLER, req);
+ } catch (ServiceException e) {
+ throw ProtobufHelper.getRemoteException(e);
+ }
+ }
+
+ private QJournalProtocolProtos.RequestInfoProto convert(
+ RequestInfo reqInfo) {
+ return QJournalProtocolProtos.RequestInfoProto.newBuilder()
+ .setJournalId(convertJournalId(reqInfo.getJournalId()))
+ .setEpoch(reqInfo.getEpoch())
+ .setIpcSerialNumber(reqInfo.getIpcSerialNumber())
+ .build();
+ }
+
+ @Override
+ public void startLogSegment(RequestInfo reqInfo, long txid)
+ throws IOException {
+ StartLogSegmentRequestProto req = StartLogSegmentRequestProto.newBuilder()
+ .setReqInfo(convert(reqInfo))
+ .setTxid(txid)
+ .build();
+ try {
+ rpcProxy.startLogSegment(NULL_CONTROLLER, req);
+ } catch (ServiceException e) {
+ throw ProtobufHelper.getRemoteException(e);
+ }
+ }
+
+ @Override
+ public void finalizeLogSegment(RequestInfo reqInfo, long startTxId,
+ long endTxId) throws IOException {
+ FinalizeLogSegmentRequestProto req =
+ FinalizeLogSegmentRequestProto.newBuilder()
+ .setReqInfo(convert(reqInfo))
+ .setStartTxId(startTxId)
+ .setEndTxId(endTxId)
+ .build();
+ try {
+ rpcProxy.finalizeLogSegment(NULL_CONTROLLER, req);
+ } catch (ServiceException e) {
+ throw ProtobufHelper.getRemoteException(e);
+ }
+ }
+
+ @Override
+ public GetEditLogManifestResponseProto getEditLogManifest(String jid,
+ long sinceTxId) throws IOException {
+ try {
+ return rpcProxy.getEditLogManifest(NULL_CONTROLLER,
+ GetEditLogManifestRequestProto.newBuilder()
+ .setJid(convertJournalId(jid))
+ .setSinceTxId(sinceTxId)
+ .build());
+ } catch (ServiceException e) {
+ throw ProtobufHelper.getRemoteException(e);
+ }
+ }
+
+ @Override
+ public PrepareRecoveryResponseProto prepareRecovery(RequestInfo reqInfo,
+ long segmentTxId) throws IOException {
+ try {
+ return rpcProxy.prepareRecovery(NULL_CONTROLLER,
+ PrepareRecoveryRequestProto.newBuilder()
+ .setReqInfo(convert(reqInfo))
+ .setSegmentTxId(segmentTxId)
+ .build());
+ } catch (ServiceException e) {
+ throw ProtobufHelper.getRemoteException(e);
+ }
+ }
+
+ @Override
+ public void acceptRecovery(RequestInfo reqInfo,
+ SegmentStateProto stateToAccept, URL fromUrl) throws IOException {
+ try {
+ rpcProxy.acceptRecovery(NULL_CONTROLLER,
+ AcceptRecoveryRequestProto.newBuilder()
+ .setReqInfo(convert(reqInfo))
+ .setStateToAccept(stateToAccept)
+ .setFromURL(fromUrl.toExternalForm())
+ .build());
+ } catch (ServiceException e) {
+ throw ProtobufHelper.getRemoteException(e);
+ }
+ }
+
+ public boolean isMethodSupported(String methodName) throws IOException {
+ return RpcClientUtil.isMethodSupported(rpcProxy,
+ QJournalProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+ RPC.getProtocolVersion(QJournalProtocolPB.class), methodName);
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/GetJournalEditServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/GetJournalEditServlet.java
new file mode 100644
index 00000000000..e1f393d721f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/GetJournalEditServlet.java
@@ -0,0 +1,208 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.server;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.hdfs.server.namenode.FileJournalManager;
+import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
+import org.apache.hadoop.hdfs.server.namenode.GetImageServlet;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.util.DataTransferThrottler;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ServletUtil;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * This servlet is used in two cases:
+ *
+ *
The QuorumJournalManager, when reading edits, fetches the edit streams
+ * from the journal nodes.
+ *
During edits synchronization, one journal node will fetch edits from
+ * another journal node.
+ *
+ */
+@InterfaceAudience.Private
+public class GetJournalEditServlet extends HttpServlet {
+
+ private static final long serialVersionUID = -4635891628211723009L;
+ private static final Log LOG = LogFactory.getLog(GetJournalEditServlet.class);
+
+ static final String STORAGEINFO_PARAM = "storageInfo";
+ static final String JOURNAL_ID_PARAM = "jid";
+ static final String SEGMENT_TXID_PARAM = "segmentTxId";
+
+ // TODO: create security tests
+ protected boolean isValidRequestor(String remoteUser, Configuration conf)
+ throws IOException {
+ if (remoteUser == null) { // This really shouldn't happen...
+ LOG.warn("Received null remoteUser while authorizing access to " +
+ "GetJournalEditServlet");
+ return false;
+ }
+
+ String[] validRequestors = {
+ SecurityUtil.getServerPrincipal(conf
+ .get(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY), NameNode
+ .getAddress(conf).getHostName()),
+ SecurityUtil.getServerPrincipal(conf
+ .get(DFSConfigKeys.DFS_JOURNALNODE_USER_NAME_KEY),
+ NameNode.getAddress(conf).getHostName()) };
+ // TODO: above principal is not correct, since each JN will have a
+ // different hostname.
+
+ for (String v : validRequestors) {
+ if (v != null && v.equals(remoteUser)) {
+ if (LOG.isDebugEnabled())
+ LOG.debug("isValidRequestor is allowing: " + remoteUser);
+ return true;
+ }
+ }
+ if (LOG.isDebugEnabled())
+ LOG.debug("isValidRequestor is rejecting: " + remoteUser);
+ return false;
+ }
+
+ private boolean checkRequestorOrSendError(Configuration conf,
+ HttpServletRequest request, HttpServletResponse response)
+ throws IOException {
+ if (UserGroupInformation.isSecurityEnabled()
+ && !isValidRequestor(request.getRemoteUser(), conf)) {
+ response.sendError(HttpServletResponse.SC_FORBIDDEN,
+ "Only Namenode and another JournalNode may access this servlet");
+ LOG.warn("Received non-NN/JN request for edits from "
+ + request.getRemoteHost());
+ return false;
+ }
+ return true;
+ }
+
+ private boolean checkStorageInfoOrSendError(JNStorage storage,
+ HttpServletRequest request, HttpServletResponse response)
+ throws IOException {
+ String myStorageInfoString = storage.toColonSeparatedString();
+ String theirStorageInfoString = request.getParameter(STORAGEINFO_PARAM);
+
+ if (theirStorageInfoString != null
+ && !myStorageInfoString.equals(theirStorageInfoString)) {
+ response.sendError(HttpServletResponse.SC_FORBIDDEN,
+ "This node has storage info " + myStorageInfoString
+ + " but the requesting node expected "
+ + theirStorageInfoString);
+ LOG.warn("Received an invalid request file transfer request "
+ + " with storage info " + theirStorageInfoString);
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public void doGet(final HttpServletRequest request,
+ final HttpServletResponse response) throws ServletException, IOException {
+ try {
+ final ServletContext context = getServletContext();
+ final Configuration conf = (Configuration) getServletContext()
+ .getAttribute(JspHelper.CURRENT_CONF);
+ final String journalId = request.getParameter(JOURNAL_ID_PARAM);
+ QuorumJournalManager.checkJournalId(journalId);
+ final JNStorage storage = JournalNodeHttpServer
+ .getJournalFromContext(context, journalId).getStorage();
+
+ // Check security
+ if (!checkRequestorOrSendError(conf, request, response)) {
+ return;
+ }
+
+ // Check that the namespace info is correct
+ if (!checkStorageInfoOrSendError(storage, request, response)) {
+ return;
+ }
+
+ long segmentTxId = ServletUtil.parseLongParam(request,
+ SEGMENT_TXID_PARAM);
+
+ FileJournalManager fjm = storage.getJournalManager();
+ File editFile;
+ FileInputStream editFileIn;
+
+ synchronized (fjm) {
+ // Synchronize on the FJM so that the file doesn't get finalized
+ // out from underneath us while we're in the process of opening
+ // it up.
+ EditLogFile elf = fjm.getLogFile(
+ segmentTxId);
+ if (elf == null) {
+ response.sendError(HttpServletResponse.SC_NOT_FOUND,
+ "No edit log found starting at txid " + segmentTxId);
+ return;
+ }
+ editFile = elf.getFile();
+ GetImageServlet.setVerificationHeaders(response, editFile);
+ GetImageServlet.setFileNameHeaders(response, editFile);
+ editFileIn = new FileInputStream(editFile);
+ }
+
+ DataTransferThrottler throttler = GetImageServlet.getThrottler(conf);
+
+ // send edits
+ TransferFsImage.getFileServer(response, editFile, editFileIn, throttler);
+
+ } catch (Throwable t) {
+ String errMsg = "getedit failed. " + StringUtils.stringifyException(t);
+ response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, errMsg);
+ throw new IOException(errMsg);
+ }
+ }
+
+ public static String buildPath(String journalId, long segmentTxId,
+ NamespaceInfo nsInfo) {
+ StringBuilder path = new StringBuilder("/getJournal?");
+ try {
+ path.append(JOURNAL_ID_PARAM).append("=")
+ .append(URLEncoder.encode(journalId, "UTF-8"));
+ path.append("&" + SEGMENT_TXID_PARAM).append("=")
+ .append(segmentTxId);
+ path.append("&" + STORAGEINFO_PARAM).append("=")
+ .append(URLEncoder.encode(nsInfo.toColonSeparatedString(), "UTF-8"));
+ } catch (UnsupportedEncodingException e) {
+ // Never get here -- everyone supports UTF-8
+ throw new RuntimeException(e);
+ }
+ return path.toString();
+ }
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java
new file mode 100644
index 00000000000..55a132a4b53
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java
@@ -0,0 +1,160 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.server;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
+import org.apache.hadoop.hdfs.server.namenode.FileJournalManager;
+import org.apache.hadoop.hdfs.server.namenode.NNStorage;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+
+/**
+ * A {@link Storage} implementation for the {@link JournalNode}.
+ *
+ * The JN has a storage directory for each namespace for which it stores
+ * metadata. There is only a single directory per JN in the current design.
+ */
+class JNStorage extends Storage {
+
+ private final FileJournalManager fjm;
+ private final StorageDirectory sd;
+ private boolean lazyInitted = false;
+
+ /**
+ * @param logDir the path to the directory in which data will be stored
+ * @param errorReporter a callback to report errors
+ */
+ protected JNStorage(File logDir, StorageErrorReporter errorReporter) {
+ super(NodeType.JOURNAL_NODE);
+
+ sd = new StorageDirectory(logDir);
+ this.addStorageDir(sd);
+ this.fjm = new FileJournalManager(sd, errorReporter);
+ }
+
+ FileJournalManager getJournalManager() {
+ return fjm;
+ }
+
+ @Override
+ public boolean isPreUpgradableLayout(StorageDirectory sd) throws IOException {
+ return false;
+ }
+
+ /**
+ * Find an edits file spanning the given transaction ID range.
+ * If no such file exists, an exception is thrown.
+ */
+ File findFinalizedEditsFile(long startTxId, long endTxId) throws IOException {
+ File ret = new File(sd.getCurrentDir(),
+ NNStorage.getFinalizedEditsFileName(startTxId, endTxId));
+ if (!ret.exists()) {
+ throw new IOException(
+ "No edits file for range " + startTxId + "-" + endTxId);
+ }
+ return ret;
+ }
+
+ /**
+ * @return the path for an in-progress edits file starting at the given
+ * transaction ID. This does not verify existence of the file.
+ */
+ File getInProgressEditLog(long startTxId) {
+ return new File(sd.getCurrentDir(),
+ NNStorage.getInProgressEditsFileName(startTxId));
+ }
+
+ /**
+ * @return the path for the file which contains persisted data for the
+ * paxos-like recovery process for the given log segment.
+ */
+ File getPaxosFile(long segmentTxId) {
+ return new File(getPaxosDir(), String.valueOf(segmentTxId));
+ }
+
+ private File getPaxosDir() {
+ return new File(sd.getCurrentDir(), "paxos");
+ }
+
+ void format(NamespaceInfo nsInfo) throws IOException {
+ setStorageInfo(nsInfo);
+ LOG.info("Formatting journal storage directory " +
+ sd + " with nsid: " + getNamespaceID());
+ sd.clearDirectory();
+ writeProperties(sd);
+ if (!getPaxosDir().mkdirs()) {
+ throw new IOException("Could not create paxos dir: " + getPaxosDir());
+ }
+ }
+
+ void analyzeStorage(NamespaceInfo nsInfo) throws IOException {
+ if (lazyInitted) {
+ checkConsistentNamespace(nsInfo);
+ return;
+ }
+
+ StorageState state = sd.analyzeStorage(StartupOption.REGULAR, this);
+ switch (state) {
+ case NON_EXISTENT:
+ case NOT_FORMATTED:
+ format(nsInfo);
+ // In the NORMAL case below, analyzeStorage() has already locked the
+ // directory for us. But in the case that we format it, we have to
+ // lock it here.
+ // The directory is unlocked in close() when the node shuts down.
+ sd.lock();
+ break;
+ case NORMAL:
+ // Storage directory is already locked by analyzeStorage() - no
+ // need to lock it here.
+ readProperties(sd);
+ checkConsistentNamespace(nsInfo);
+ break;
+
+ default:
+ LOG.warn("TODO: unhandled state for storage dir " + sd + ": " + state);
+ }
+ lazyInitted = true;
+ }
+
+ private void checkConsistentNamespace(NamespaceInfo nsInfo)
+ throws IOException {
+ if (nsInfo.getNamespaceID() != getNamespaceID()) {
+ throw new IOException("Incompatible namespaceID for journal " +
+ this.sd + ": NameNode has nsId " + nsInfo.getNamespaceID() +
+ " but storage has nsId " + getNamespaceID());
+ }
+
+ if (!nsInfo.getClusterID().equals(getClusterID())) {
+ throw new IOException("Incompatible clusterID for journal " +
+ this.sd + ": NameNode has clusterId '" + nsInfo.getClusterID() +
+ "' but storage has clusterId '" + getClusterID() + "'");
+
+ }
+ }
+
+ public void close() throws IOException {
+ LOG.info("Closing journal storage for " + sd);
+ unlockAll();
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
new file mode 100644
index 00000000000..9db98803283
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
@@ -0,0 +1,493 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.server;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.net.URL;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PersistedRecoveryPaxosData;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
+import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
+import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
+import org.apache.hadoop.hdfs.server.namenode.FileJournalManager;
+import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
+import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.hdfs.util.AtomicFileOutputStream;
+import org.apache.hadoop.hdfs.util.PersistentLongFile;
+import org.apache.hadoop.io.IOUtils;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.TextFormat;
+
+/**
+ * A JournalNode can manage journals for several clusters at once.
+ * Each such journal is entirely independent despite being hosted by
+ * the same JVM.
+ */
+class Journal implements Closeable {
+ static final Log LOG = LogFactory.getLog(Journal.class);
+
+
+ // Current writing state
+ private EditLogOutputStream curSegment;
+ private long curSegmentTxId = HdfsConstants.INVALID_TXID;
+ private long nextTxId = HdfsConstants.INVALID_TXID;
+
+ private final JNStorage storage;
+
+ /**
+ * When a new writer comes along, it asks each node to promise
+ * to ignore requests from any previous writer, as identified
+ * by epoch number. In order to make such a promise, the epoch
+ * number of that writer is stored persistently on disk.
+ */
+ private PersistentLongFile lastPromisedEpoch;
+ private static final String LAST_PROMISED_FILENAME = "last-promised-epoch";
+
+ private final FileJournalManager fjm;
+
+ Journal(File logDir, StorageErrorReporter errorReporter) {
+ storage = new JNStorage(logDir, errorReporter);
+
+ File currentDir = storage.getSingularStorageDir().getCurrentDir();
+ this.lastPromisedEpoch = new PersistentLongFile(
+ new File(currentDir, LAST_PROMISED_FILENAME), 0);
+
+ this.fjm = storage.getJournalManager();
+ }
+
+ /**
+ * Iterate over the edit logs stored locally, and set
+ * {@link #curSegmentTxId} to refer to the most recently written
+ * one.
+ */
+ private synchronized void scanStorage() throws IOException {
+ if (!fjm.getStorageDirectory().getCurrentDir().exists()) {
+ return;
+ }
+ LOG.info("Scanning storage " + fjm);
+ List files = fjm.getLogFiles(0);
+ if (!files.isEmpty()) {
+ EditLogFile latestLog = files.get(files.size() - 1);
+ LOG.info("Latest log is " + latestLog);
+ curSegmentTxId = latestLog.getFirstTxId();
+ }
+ }
+
+ /**
+ * Format the local storage with the given namespace.
+ */
+ void format(NamespaceInfo nsInfo) throws IOException {
+ Preconditions.checkState(nsInfo.getNamespaceID() != 0,
+ "can't format with uninitialized namespace info: %s",
+ nsInfo);
+ storage.format(nsInfo);
+ }
+
+ /**
+ * Unlock and release resources.
+ */
+ @Override // Closeable
+ public void close() throws IOException {
+ storage.close();
+ }
+
+ JNStorage getStorage() {
+ return storage;
+ }
+
+ /**
+ * @return the last epoch which this node has promised not to accept
+ * any lower epoch, or 0 if no promises have been made.
+ */
+ synchronized long getLastPromisedEpoch() throws IOException {
+ return lastPromisedEpoch.get();
+ }
+
+ /**
+ * Try to create a new epoch for this journal.
+ * @param nsInfo the namespace, which is verified for consistency or used to
+ * format, if the Journal has not yet been written to.
+ * @param epoch the epoch to start
+ * @return the status information necessary to begin recovery
+ * @throws IOException if the node has already made a promise to another
+ * writer with a higher epoch number, if the namespace is inconsistent,
+ * or if a disk error occurs.
+ */
+ synchronized NewEpochResponseProto newEpoch(
+ NamespaceInfo nsInfo, long epoch) throws IOException {
+
+ // If the storage is unformatted, format it with this NS.
+ // Otherwise, check that the NN's nsinfo matches the storage.
+ storage.analyzeStorage(nsInfo);
+
+ if (epoch <= getLastPromisedEpoch()) {
+ throw new IOException("Proposed epoch " + epoch + " <= last promise " +
+ getLastPromisedEpoch());
+ }
+
+ lastPromisedEpoch.set(epoch);
+ if (curSegment != null) {
+ curSegment.close();
+ curSegment = null;
+ }
+
+ NewEpochResponseProto.Builder builder =
+ NewEpochResponseProto.newBuilder();
+
+ // TODO: we only need to do this once, not on writer switchover.
+ scanStorage();
+
+ if (curSegmentTxId != HdfsConstants.INVALID_TXID) {
+ builder.setLastSegmentTxId(curSegmentTxId);
+ }
+
+ return builder.build();
+ }
+
+ /**
+ * Write a batch of edits to the journal.
+ * {@see QJournalProtocol#journal(RequestInfo, long, int, byte[])}
+ */
+ synchronized void journal(RequestInfo reqInfo, long firstTxnId,
+ int numTxns, byte[] records) throws IOException {
+ checkRequest(reqInfo);
+
+ // TODO: if a JN goes down and comes back up, then it will throw
+ // this exception on every edit. We should instead send back
+ // a response indicating the log needs to be rolled, which would
+ // mark the logger on the client side as "pending" -- and have the
+ // NN code look for this condition and trigger a roll when it happens.
+ // That way the node can catch back up and rejoin
+ Preconditions.checkState(curSegment != null,
+ "Can't write, no segment open");
+ Preconditions.checkState(nextTxId == firstTxnId,
+ "Can't write txid " + firstTxnId + " expecting nextTxId=" + nextTxId);
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Writing txid " + firstTxnId + "-" + (firstTxnId + numTxns - 1));
+ }
+ curSegment.writeRaw(records, 0, records.length);
+ curSegment.setReadyToFlush();
+ curSegment.flush();
+ nextTxId += numTxns;
+ }
+
+ /**
+ * Ensure that the given request is coming from the correct writer and in-order.
+ * @param reqInfo the request info
+ * @throws IOException if the request is invalid.
+ */
+ private synchronized void checkRequest(RequestInfo reqInfo) throws IOException {
+ // Invariant 25 from ZAB paper
+ if (reqInfo.getEpoch() < lastPromisedEpoch.get()) {
+ throw new IOException("IPC's epoch " + reqInfo.getEpoch() +
+ " is less than the last promised epoch " +
+ lastPromisedEpoch.get());
+ }
+
+ // TODO: should other requests check the _exact_ epoch instead of
+ // the <= check? <= should probably only be necessary for the
+ // first calls
+
+ // TODO: some check on serial number that they only increase from a given
+ // client
+ }
+
+ /**
+ * Start a new segment at the given txid. The previous segment
+ * must have already been finalized.
+ */
+ public synchronized void startLogSegment(RequestInfo reqInfo, long txid)
+ throws IOException {
+ assert fjm != null;
+ checkRequest(reqInfo);
+
+ Preconditions.checkState(curSegment == null,
+ "Can't start a log segment, already writing " + curSegment);
+ Preconditions.checkState(nextTxId == txid || nextTxId == HdfsConstants.INVALID_TXID,
+ "Can't start log segment " + txid + " expecting nextTxId=" + nextTxId);
+ curSegment = fjm.startLogSegment(txid);
+ curSegmentTxId = txid;
+ nextTxId = txid;
+ }
+
+ /**
+ * Finalize the log segment at the given transaction ID.
+ */
+ public synchronized void finalizeLogSegment(RequestInfo reqInfo, long startTxId,
+ long endTxId) throws IOException {
+ checkRequest(reqInfo);
+
+ if (startTxId == curSegmentTxId) {
+ if (curSegment != null) {
+ curSegment.close();
+ curSegment = null;
+ }
+ }
+
+ FileJournalManager.EditLogFile elf = fjm.getLogFile(startTxId);
+ if (elf.isInProgress()) {
+ // TODO: this is slow to validate when in non-recovery cases
+ // we already know the length here!
+
+ LOG.info("Validating log about to be finalized: " + elf);
+ elf.validateLog();
+
+ Preconditions.checkState(elf.getLastTxId() == endTxId,
+ "Trying to finalize log %s-%s, but current state of log" +
+ "is %s", startTxId, endTxId, elf);
+ fjm.finalizeLogSegment(startTxId, endTxId);
+ } else {
+ Preconditions.checkArgument(endTxId == elf.getLastTxId(),
+ "Trying to re-finalize already finalized log " +
+ elf + " with different endTxId " + endTxId);
+ }
+ }
+
+ /**
+ * @see QJournalProtocol#getEditLogManifest(String, long)
+ */
+ public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
+ throws IOException {
+ // TODO: check fencing info?
+ RemoteEditLogManifest manifest = new RemoteEditLogManifest(
+ fjm.getRemoteEditLogs(sinceTxId));
+ return manifest;
+ }
+
+ /**
+ * @return the current state of the given segment, or null if the
+ * segment does not exist.
+ */
+ private SegmentStateProto getSegmentInfo(long segmentTxId)
+ throws IOException {
+ EditLogFile elf = fjm.getLogFile(segmentTxId);
+ if (elf == null) {
+ return null;
+ }
+ if (elf.isInProgress()) {
+ elf.validateLog();
+ }
+ if (elf.getLastTxId() == HdfsConstants.INVALID_TXID) {
+ // no transactions in file
+ throw new AssertionError("TODO: no transactions in file " +
+ elf);
+ }
+ SegmentStateProto ret = SegmentStateProto.newBuilder()
+ .setStartTxId(segmentTxId)
+ .setEndTxId(elf.getLastTxId())
+ .setIsInProgress(elf.isInProgress())
+ .setMd5Sum(ByteString.EMPTY) // TODO
+ .build();
+ LOG.info("getSegmentInfo(" + segmentTxId + "): " + elf + " -> " +
+ TextFormat.shortDebugString(ret));
+ return ret;
+ }
+
+ /**
+ * @see QJournalProtocol#prepareRecovery(RequestInfo, long)
+ */
+ public synchronized PrepareRecoveryResponseProto prepareRecovery(
+ RequestInfo reqInfo, long segmentTxId) throws IOException {
+ checkRequest(reqInfo);
+
+ PrepareRecoveryResponseProto.Builder builder =
+ PrepareRecoveryResponseProto.newBuilder();
+
+ PersistedRecoveryPaxosData previouslyAccepted = getPersistedPaxosData(segmentTxId);
+ if (previouslyAccepted != null) {
+ builder.setAcceptedInEpoch(previouslyAccepted.getAcceptedInEpoch())
+ .setSegmentState(previouslyAccepted.getSegmentState());
+ } else {
+ SegmentStateProto segInfo = getSegmentInfo(segmentTxId);
+ if (segInfo != null) {
+ builder.setSegmentState(segInfo);
+ }
+ }
+
+ PrepareRecoveryResponseProto resp = builder.build();
+ LOG.info("Prepared recovery for segment " + segmentTxId + ": " +
+ TextFormat.shortDebugString(resp));
+ return resp;
+ }
+
+ /**
+ * @see QJournalProtocol#acceptRecovery(RequestInfo, SegmentStateProto, URL)
+ */
+ public synchronized void acceptRecovery(RequestInfo reqInfo,
+ SegmentStateProto segment, URL fromUrl)
+ throws IOException {
+ checkRequest(reqInfo);
+ long segmentTxId = segment.getStartTxId();
+
+ // TODO: right now, a recovery of a segment when the log is
+ // completely emtpy (ie startLogSegment() but no txns)
+ // will fail this assertion here, since endTxId < startTxId
+ Preconditions.checkArgument(segment.getEndTxId() > 0 &&
+ segment.getEndTxId() >= segmentTxId,
+ "bad recovery state for segment %s: %s",
+ segmentTxId, TextFormat.shortDebugString(segment));
+
+ PersistedRecoveryPaxosData oldData = getPersistedPaxosData(segmentTxId);
+ PersistedRecoveryPaxosData newData = PersistedRecoveryPaxosData.newBuilder()
+ .setAcceptedInEpoch(reqInfo.getEpoch())
+ .setSegmentState(segment)
+ .build();
+ if (oldData != null) {
+ Preconditions.checkState(oldData.getAcceptedInEpoch() <= reqInfo.getEpoch(),
+ "Bad paxos transition, out-of-order epochs.\nOld: %s\nNew: %s\n",
+ oldData, newData);
+ }
+
+ SegmentStateProto currentSegment = getSegmentInfo(segmentTxId);
+ // TODO: this can be null, in the case that one of the loggers started
+ // the next segment, but others did not! add regression test and null
+ // check in next condition below.
+
+ // TODO: what if they have the same length but one is finalized and the
+ // other isn't! cover that case.
+ if (currentSegment.getEndTxId() != segment.getEndTxId()) {
+ syncLog(reqInfo, segment, fromUrl);
+ } else {
+ LOG.info("Skipping download of log " +
+ TextFormat.shortDebugString(segment) +
+ ": already have up-to-date logs");
+ }
+
+ // TODO: is it OK that this is non-atomic?
+ // we might be left with an older epoch recorded, but a newer log
+
+ persistPaxosData(segmentTxId, newData);
+ LOG.info("Accepted recovery for segment " + segmentTxId + ": " +
+ TextFormat.shortDebugString(newData));
+ }
+
+ /**
+ * Synchronize a log segment from another JournalNode.
+ * @param reqInfo the request info for the recovery IPC
+ * @param segment
+ * @param url
+ * @throws IOException
+ */
+ private void syncLog(RequestInfo reqInfo,
+ SegmentStateProto segment, URL url) throws IOException {
+ String tmpFileName =
+ "synclog_" + segment.getStartTxId() + "_" +
+ reqInfo.getEpoch() + "." + reqInfo.getIpcSerialNumber();
+
+ List localPaths = storage.getFiles(null, tmpFileName);
+ assert localPaths.size() == 1;
+ File tmpFile = localPaths.get(0);
+
+ boolean success = false;
+
+ LOG.info("Synchronizing log " +
+ TextFormat.shortDebugString(segment) + " from " + url);
+ TransferFsImage.doGetUrl(url, localPaths, storage, true);
+ assert tmpFile.exists();
+ try {
+ success = tmpFile.renameTo(storage.getInProgressEditLog(
+ segment.getStartTxId()));
+ if (success) {
+ // If we're synchronizing the latest segment, update our cached
+ // info.
+ // TODO: can this be done more generally?
+ if (curSegmentTxId == segment.getStartTxId()) {
+ nextTxId = segment.getEndTxId() + 1;
+ }
+ }
+ } finally {
+ if (!success) {
+ if (!tmpFile.delete()) {
+ LOG.warn("Failed to delete temporary file " + tmpFile);
+ }
+ }
+ }
+ }
+
+ /**
+ * Retrieve the persisted data for recovering the given segment from disk.
+ */
+ private PersistedRecoveryPaxosData getPersistedPaxosData(long segmentTxId)
+ throws IOException {
+ File f = storage.getPaxosFile(segmentTxId);
+ if (!f.exists()) {
+ // Default instance has no fields filled in (they're optional)
+ return null;
+ }
+
+ InputStream in = new FileInputStream(f);
+ try {
+ PersistedRecoveryPaxosData ret = PersistedRecoveryPaxosData.parseDelimitedFrom(in);
+ Preconditions.checkState(ret != null &&
+ ret.getSegmentState().getStartTxId() == segmentTxId,
+ "Bad persisted data for segment %s: %s",
+ segmentTxId, ret);
+ return ret;
+ } finally {
+ IOUtils.closeStream(in);
+ }
+ }
+
+ /**
+ * Persist data for recovering the given segment from disk.
+ */
+ private void persistPaxosData(long segmentTxId,
+ PersistedRecoveryPaxosData newData) throws IOException {
+ File f = storage.getPaxosFile(segmentTxId);
+ boolean success = false;
+ AtomicFileOutputStream fos = new AtomicFileOutputStream(f);
+ try {
+ newData.writeDelimitedTo(fos);
+ fos.write('\n');
+ // Write human-readable data after the protobuf. This is only
+ // to assist in debugging -- it's not parsed at all.
+ OutputStreamWriter writer = new OutputStreamWriter(fos);
+
+ writer.write(String.valueOf(newData));
+ writer.write('\n');
+ writer.flush();
+
+ fos.flush();
+ success = true;
+ } finally {
+ if (success) {
+ IOUtils.closeStream(fos);
+ } else {
+ fos.abort();
+ }
+ }
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
new file mode 100644
index 00000000000..46540b7833d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
@@ -0,0 +1,202 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.server;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
+import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.source.JvmMetrics;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+
+/**
+ * The JournalNode is a daemon which allows namenodes using
+ * the QuorumJournalManager to log and retrieve edits stored
+ * remotely. It is a thin wrapper around a local edit log
+ * directory with the addition of facilities to participate
+ * in the quorum protocol.
+ */
+@InterfaceAudience.Private
+public class JournalNode implements Tool, Configurable {
+ public static final Log LOG = LogFactory.getLog(JournalNode.class);
+ private Configuration conf;
+ private JournalNodeRpcServer rpcServer;
+ private JournalNodeHttpServer httpServer;
+ private Map journalsById = Maps.newHashMap();
+
+ static {
+ HdfsConfiguration.init();
+ }
+
+ /**
+ * When stopped, the daemon will exit with this code.
+ */
+ private int resultCode = 0;
+
+ synchronized Journal getOrCreateJournal(String jid) {
+ QuorumJournalManager.checkJournalId(jid);
+
+ Journal journal = journalsById.get(jid);
+ if (journal == null) {
+ File logDir = getLogDir(jid);
+ LOG.info("Initializing journal in directory " + logDir);
+ journal = new Journal(logDir, new ErrorReporter());
+ journalsById.put(jid, journal);
+ }
+
+ return journal;
+ }
+
+
+ @Override
+ public void setConf(Configuration conf) {
+ this.conf = conf;
+ }
+
+ @Override
+ public Configuration getConf() {
+ return conf;
+ }
+
+ @Override
+ public int run(String[] args) throws Exception {
+ start();
+ return join();
+ }
+
+ /**
+ * Start listening for edits via RPC.
+ */
+ public void start() throws IOException {
+ Preconditions.checkState(!isStarted(), "JN already running");
+
+ DefaultMetricsSystem.initialize("JournalNode");
+ JvmMetrics.create("JournalNode",
+ conf.get(DFSConfigKeys.DFS_METRICS_SESSION_ID_KEY),
+ DefaultMetricsSystem.instance());
+
+ httpServer = new JournalNodeHttpServer(conf, this);
+ httpServer.start();
+
+ rpcServer = new JournalNodeRpcServer(conf, this);
+ rpcServer.start();
+ }
+
+ public boolean isStarted() {
+ return rpcServer != null;
+ }
+
+ /**
+ * @return the address the IPC server is bound to
+ */
+ public InetSocketAddress getBoundIpcAddress() {
+ return rpcServer.getAddress();
+ }
+
+
+ public InetSocketAddress getBoundHttpAddress() {
+ return httpServer.getAddress();
+ }
+
+
+ /**
+ * Stop the daemon with the given status code
+ * @param rc the status code with which to exit (non-zero
+ * should indicate an error)
+ */
+ public void stop(int rc) {
+ this.resultCode = rc;
+
+ if (rpcServer != null) {
+ rpcServer.stop();
+ }
+
+ if (httpServer != null) {
+ try {
+ httpServer.stop();
+ } catch (IOException ioe) {
+ LOG.warn("Unable to stop HTTP server for " + this, ioe);
+ }
+ }
+
+ for (Journal j : journalsById.values()) {
+ IOUtils.cleanup(LOG, j);
+ }
+ }
+
+ /**
+ * Wait for the daemon to exit.
+ * @return the result code (non-zero if error)
+ */
+ int join() throws InterruptedException {
+ if (rpcServer != null) {
+ rpcServer.join();
+ }
+ return resultCode;
+ }
+
+ public void stopAndJoin(int rc) throws InterruptedException {
+ stop(rc);
+ join();
+ }
+
+ /**
+ * Return the directory inside our configured storage
+ * dir which corresponds to a given journal.
+ * @param jid the journal identifier
+ * @return the file, which may or may not exist yet
+ */
+ private File getLogDir(String jid) {
+ String dir = conf.get(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY,
+ DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_DEFAULT);
+ Preconditions.checkArgument(jid != null &&
+ !jid.isEmpty(),
+ "bad journal identifier: %s", jid);
+ return new File(new File(dir), jid);
+ }
+
+
+ private class ErrorReporter implements StorageErrorReporter {
+ @Override
+ public void reportErrorOnFile(File f) {
+ LOG.fatal("Error reported on file " + f + "... exiting",
+ new Exception());
+ stop(1);
+ }
+ }
+
+ public static void main(String[] args) throws Exception {
+ System.exit(ToolRunner.run(new JournalNode(), args));
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeHttpServer.java
new file mode 100644
index 00000000000..6b15b0b9fd8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeHttpServer.java
@@ -0,0 +1,126 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.server;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_JOURNALNODE_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_JOURNALNODE_USER_NAME_KEY;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import javax.servlet.ServletContext;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
+
+/**
+ * Encapsulates the HTTP server started by the Journal Service.
+ */
+@InterfaceAudience.Private
+public class JournalNodeHttpServer {
+ public static final Log LOG = LogFactory.getLog(
+ JournalNodeHttpServer.class);
+
+ public static final String JN_ATTRIBUTE_KEY = "localjournal";
+
+ private HttpServer httpServer;
+ private int infoPort;
+ private JournalNode localJournalNode;
+
+ private final Configuration conf;
+
+ JournalNodeHttpServer(Configuration conf, JournalNode jn) {
+ this.conf = conf;
+ this.localJournalNode = jn;
+ }
+
+ void start() throws IOException {
+ final InetSocketAddress bindAddr = getAddress(conf);
+
+ // initialize the webserver for uploading/downloading files.
+ LOG.info("Starting web server as: "
+ + UserGroupInformation.getCurrentUser().getUserName());
+
+ int tmpInfoPort = bindAddr.getPort();
+ httpServer = new HttpServer("journal", bindAddr.getHostName(),
+ tmpInfoPort, tmpInfoPort == 0, conf, new AccessControlList(conf
+ .get(DFS_ADMIN, " "))) {
+ {
+ if (UserGroupInformation.isSecurityEnabled()) {
+ initSpnego(conf, DFS_JOURNALNODE_USER_NAME_KEY,
+ DFS_JOURNALNODE_KEYTAB_FILE_KEY);
+ }
+ }
+ };
+ httpServer.setAttribute(JN_ATTRIBUTE_KEY, localJournalNode);
+ httpServer.setAttribute(JspHelper.CURRENT_CONF, conf);
+ httpServer.addInternalServlet("getJournal", "/getJournal",
+ GetJournalEditServlet.class, true);
+ httpServer.start();
+
+ // The web-server port can be ephemeral... ensure we have the correct info
+ infoPort = httpServer.getPort();
+
+ LOG.info("Journal Web-server up at: " + bindAddr + ":" + infoPort);
+ }
+
+ void stop() throws IOException {
+ if (httpServer != null) {
+ try {
+ httpServer.stop();
+ } catch (Exception e) {
+ throw new IOException(e);
+ }
+ }
+ }
+
+ /**
+ * Return the actual address bound to by the running server.
+ */
+ public InetSocketAddress getAddress() {
+ InetSocketAddress addr = httpServer.getListenerAddress();
+ assert addr.getPort() != 0;
+ return addr;
+ }
+
+ private static InetSocketAddress getAddress(Configuration conf) {
+ String addr = conf.get(DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_KEY,
+ DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_DEFAULT);
+ return NetUtils.createSocketAddr(addr,
+ DFSConfigKeys.DFS_JOURNALNODE_HTTP_PORT_DEFAULT,
+ DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_KEY);
+ }
+
+ public static Journal getJournalFromContext(ServletContext context, String jid) {
+ JournalNode jn = (JournalNode)context.getAttribute(JN_ATTRIBUTE_KEY);
+ return jn.getOrCreateJournal(jid);
+ }
+
+ public static Configuration getConfFromContext(ServletContext context) {
+ return (Configuration) context.getAttribute(JspHelper.CURRENT_CONF);
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
new file mode 100644
index 00000000000..7a7173bc3e5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
@@ -0,0 +1,160 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.server;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URL;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+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.GetJournalStateResponseProto;
+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.QJournalProtocolService;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
+import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolPB;
+import org.apache.hadoop.hdfs.qjournal.protocolPB.QJournalProtocolServerSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RPC.Server;
+import org.apache.hadoop.net.NetUtils;
+
+import com.google.protobuf.BlockingService;
+
+class JournalNodeRpcServer implements QJournalProtocol {
+
+ private static final int HANDLER_COUNT = 5;
+ private JournalNode jn;
+ private Server server;
+
+ JournalNodeRpcServer(Configuration conf, JournalNode jn) throws IOException {
+ this.jn = jn;
+
+ InetSocketAddress addr = getAddress(conf);
+ RPC.setProtocolEngine(conf, QJournalProtocolPB.class,
+ ProtobufRpcEngine.class);
+ QJournalProtocolServerSideTranslatorPB translator =
+ new QJournalProtocolServerSideTranslatorPB(this);
+ BlockingService service = QJournalProtocolService
+ .newReflectiveBlockingService(translator);
+
+ this.server = RPC.getServer(
+ QJournalProtocolPB.class,
+ service, addr.getHostName(),
+ addr.getPort(), HANDLER_COUNT, false, conf,
+ null /*secretManager*/);
+ }
+
+ void start() {
+ this.server.start();
+ }
+
+ public InetSocketAddress getAddress() {
+ return server.getListenerAddress();
+ }
+
+ void join() throws InterruptedException {
+ this.server.join();
+ }
+
+ void stop() {
+ this.server.stop();
+ }
+
+ private static InetSocketAddress getAddress(Configuration conf) {
+ String addr = conf.get(
+ DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_KEY,
+ DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_DEFAULT);
+ return NetUtils.createSocketAddr(addr, 0,
+ DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_KEY);
+ }
+
+ @Override
+ public GetJournalStateResponseProto getJournalState(String journalId)
+ throws IOException {
+ long epoch = jn.getOrCreateJournal(journalId).getLastPromisedEpoch();
+ return GetJournalStateResponseProto.newBuilder()
+ .setLastPromisedEpoch(epoch)
+ .setHttpPort(jn.getBoundHttpAddress().getPort())
+ .build();
+ }
+
+ @Override
+ public NewEpochResponseProto newEpoch(String journalId,
+ NamespaceInfo nsInfo,
+ long epoch) throws IOException {
+ return jn.getOrCreateJournal(journalId).newEpoch(nsInfo, epoch);
+ }
+
+
+ @Override
+ public void journal(RequestInfo reqInfo, long firstTxnId,
+ int numTxns, byte[] records) throws IOException {
+ jn.getOrCreateJournal(reqInfo.getJournalId())
+ .journal(reqInfo, firstTxnId, numTxns, records);
+ }
+
+ @Override
+ public void startLogSegment(RequestInfo reqInfo, long txid)
+ throws IOException {
+ jn.getOrCreateJournal(reqInfo.getJournalId())
+ .startLogSegment(reqInfo, txid);
+ }
+
+ @Override
+ public void finalizeLogSegment(RequestInfo reqInfo, long startTxId,
+ long endTxId) throws IOException {
+ jn.getOrCreateJournal(reqInfo.getJournalId())
+ .finalizeLogSegment(reqInfo, startTxId, endTxId);
+ }
+
+ @Override
+ public GetEditLogManifestResponseProto getEditLogManifest(String jid,
+ long sinceTxId) throws IOException {
+
+ RemoteEditLogManifest manifest = jn.getOrCreateJournal(jid)
+ .getEditLogManifest(sinceTxId);
+
+ return GetEditLogManifestResponseProto.newBuilder()
+ .setManifest(PBHelper.convert(manifest))
+ .setHttpPort(jn.getBoundHttpAddress().getPort())
+ .build();
+ }
+
+ @Override
+ public PrepareRecoveryResponseProto prepareRecovery(RequestInfo reqInfo,
+ long segmentTxId) throws IOException {
+ return jn.getOrCreateJournal(reqInfo.getJournalId())
+ .prepareRecovery(reqInfo, segmentTxId);
+ }
+
+ @Override
+ public void acceptRecovery(RequestInfo reqInfo, SegmentStateProto log,
+ URL fromUrl) throws IOException {
+ jn.getOrCreateJournal(reqInfo.getJournalId())
+ .acceptRecovery(reqInfo, log, fromUrl);
+ }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index 993869512d0..532ef1a8f42 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -39,7 +39,8 @@ public final class HdfsServerConstants {
*/
static public enum NodeType {
NAME_NODE,
- DATA_NODE;
+ DATA_NODE,
+ JOURNAL_NODE;
}
/** Startup options */
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
index c5db2d3e3f7..70692d14210 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
@@ -41,6 +41,8 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.util.VersionInfo;
+import com.google.common.base.Preconditions;
+
/**
@@ -75,7 +77,7 @@ public abstract class Storage extends StorageInfo {
/** Layout versions of 0.20.203 release */
public static final int[] LAYOUT_VERSIONS_203 = {-19, -31};
- private static final String STORAGE_FILE_LOCK = "in_use.lock";
+ public static final String STORAGE_FILE_LOCK = "in_use.lock";
protected static final String STORAGE_FILE_VERSION = "VERSION";
public static final String STORAGE_DIR_CURRENT = "current";
public static final String STORAGE_DIR_PREVIOUS = "previous";
@@ -719,6 +721,15 @@ public abstract class Storage extends StorageInfo {
return storageDirs.get(idx);
}
+ /**
+ * @return the storage directory, with the precondition that this storage
+ * has exactly one storage directory
+ */
+ public StorageDirectory getSingularStorageDir() {
+ Preconditions.checkState(storageDirs.size() == 1);
+ return storageDirs.get(0);
+ }
+
protected void addStorageDir(StorageDirectory sd) {
storageDirs.add(sd);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java
index e9d22e30bd7..d6c8a84741b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import java.io.OutputStream;
+import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.Writer;
import org.apache.hadoop.io.DataOutputBuffer;
@@ -34,7 +35,8 @@ import com.google.common.base.Preconditions;
* to progress concurrently to flushes without allocating new buffers each
* time.
*/
-class EditsDoubleBuffer {
+@InterfaceAudience.Private
+public class EditsDoubleBuffer {
private TxnBuffer bufCurrent; // current buffer for writing
private TxnBuffer bufReady; // buffer ready for flushing
@@ -51,11 +53,11 @@ class EditsDoubleBuffer {
bufCurrent.writeOp(op);
}
- void writeRaw(byte[] bytes, int offset, int length) throws IOException {
+ public void writeRaw(byte[] bytes, int offset, int length) throws IOException {
bufCurrent.write(bytes, offset, length);
}
- void close() throws IOException {
+ public void close() throws IOException {
Preconditions.checkNotNull(bufCurrent);
Preconditions.checkNotNull(bufReady);
@@ -69,7 +71,7 @@ class EditsDoubleBuffer {
bufCurrent = bufReady = null;
}
- void setReadyToFlush() {
+ public void setReadyToFlush() {
assert isFlushed() : "previous data not flushed yet";
TxnBuffer tmp = bufReady;
bufReady = bufCurrent;
@@ -80,12 +82,12 @@ class EditsDoubleBuffer {
* Writes the content of the "ready" buffer to the given output stream,
* and resets it. Does not swap any buffers.
*/
- void flushTo(OutputStream out) throws IOException {
+ public void flushTo(OutputStream out) throws IOException {
bufReady.writeTo(out); // write data to file
bufReady.reset(); // erase all data in the buffer
}
- boolean shouldForceSync() {
+ public boolean shouldForceSync() {
return bufCurrent.size() >= initBufferSize;
}
@@ -120,6 +122,12 @@ class EditsDoubleBuffer {
return bufReady.numTxns;
}
+ /**
+ * @return the number of bytes that are ready to be flushed
+ */
+ public int countReadyBytes() {
+ return bufReady.size();
+ }
private static class TxnBuffer extends DataOutputBuffer {
long firstTxId;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index 6a549f788b7..1bd8f0db16a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -1136,6 +1136,7 @@ public class FSEditLog {
journalSet.recoverUnfinalizedSegments();
} catch (IOException ex) {
// All journals have failed, it is handled in logSync.
+ // TODO: are we sure this is OK?
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
index ff1b2545420..1527c676207 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
@@ -29,6 +29,7 @@ import java.util.Collections;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
+import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -39,6 +40,7 @@ import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.ComparisonChain;
@@ -50,7 +52,8 @@ import com.google.common.collect.ComparisonChain;
* Note: this class is not thread-safe and should be externally
* synchronized.
*/
-class FileJournalManager implements JournalManager {
+@InterfaceAudience.Private
+public class FileJournalManager implements JournalManager {
private static final Log LOG = LogFactory.getLog(FileJournalManager.class);
private final StorageDirectory sd;
@@ -147,7 +150,7 @@ class FileJournalManager implements JournalManager {
* @return a list of remote edit logs
* @throws IOException if edit logs cannot be listed.
*/
- List getRemoteEditLogs(long firstTxId) throws IOException {
+ public List getRemoteEditLogs(long firstTxId) throws IOException {
File currentDir = sd.getCurrentDir();
List allLogFiles = matchEditLogs(currentDir);
List ret = Lists.newArrayListWithCapacity(
@@ -165,6 +168,8 @@ class FileJournalManager implements JournalManager {
}
}
+ Collections.sort(ret);
+
return ret;
}
@@ -178,7 +183,7 @@ class FileJournalManager implements JournalManager {
* @throws IOException
* IOException thrown for invalid logDir
*/
- static List matchEditLogs(File logDir) throws IOException {
+ public static List matchEditLogs(File logDir) throws IOException {
return matchEditLogs(FileUtil.listFiles(logDir));
}
@@ -206,7 +211,7 @@ class FileJournalManager implements JournalManager {
try {
long startTxId = Long.valueOf(inProgressEditsMatch.group(1));
ret.add(
- new EditLogFile(f, startTxId, startTxId, true));
+ new EditLogFile(f, startTxId, HdfsConstants.INVALID_TXID, true));
} catch (NumberFormatException nfe) {
LOG.error("In-progress edits file " + f + " has improperly " +
"formatted transaction ID");
@@ -304,7 +309,7 @@ class FileJournalManager implements JournalManager {
}
}
- List getLogFiles(long fromTxId) throws IOException {
+ public List getLogFiles(long fromTxId) throws IOException {
File currentDir = sd.getCurrentDir();
List allLogFiles = matchEditLogs(currentDir);
List logFiles = Lists.newArrayList();
@@ -320,6 +325,32 @@ class FileJournalManager implements JournalManager {
return logFiles;
}
+
+ public EditLogFile getLogFile(long startTxId) throws IOException {
+ return getLogFile(sd.getCurrentDir(), startTxId);
+ }
+
+ public static EditLogFile getLogFile(File dir, long startTxId)
+ throws IOException {
+ List files = matchEditLogs(dir);
+ List ret = Lists.newLinkedList();
+ for (EditLogFile elf : files) {
+ if (elf.getFirstTxId() == startTxId) {
+ ret.add(elf);
+ }
+ }
+
+ if (ret.isEmpty()) {
+ // no matches
+ return null;
+ } else if (ret.size() == 1) {
+ return ret.get(0);
+ } else {
+ throw new IllegalStateException("More than one log segment in " +
+ dir + " starting at txid " + startTxId + ": " +
+ Joiner.on(", ").join(ret));
+ }
+ }
@Override
public String toString() {
@@ -329,7 +360,8 @@ class FileJournalManager implements JournalManager {
/**
* Record of an edit log that has been located and had its filename parsed.
*/
- static class EditLogFile {
+ @InterfaceAudience.Private
+ public static class EditLogFile {
private File file;
private final long firstTxId;
private long lastTxId;
@@ -362,17 +394,20 @@ class FileJournalManager implements JournalManager {
assert (firstTxId > 0) || (firstTxId == HdfsConstants.INVALID_TXID);
assert file != null;
+ Preconditions.checkArgument(!isInProgress ||
+ lastTxId == HdfsConstants.INVALID_TXID);
+
this.firstTxId = firstTxId;
this.lastTxId = lastTxId;
this.file = file;
this.isInProgress = isInProgress;
}
- long getFirstTxId() {
+ public long getFirstTxId() {
return firstTxId;
}
- long getLastTxId() {
+ public long getLastTxId() {
return lastTxId;
}
@@ -385,17 +420,17 @@ class FileJournalManager implements JournalManager {
* This will update the lastTxId of the EditLogFile or
* mark it as corrupt if it is.
*/
- void validateLog() throws IOException {
+ public void validateLog() throws IOException {
EditLogValidation val = EditLogFileInputStream.validateEditLog(file);
this.lastTxId = val.getEndTxId();
this.hasCorruptHeader = val.hasCorruptHeader();
}
- boolean isInProgress() {
+ public boolean isInProgress() {
return isInProgress;
}
- File getFile() {
+ public File getFile() {
return file;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
index 3c67078c724..e3e3968e435 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
@@ -229,6 +229,13 @@ public class JournalSet implements JournalManager {
}
jas.getManager().selectInputStreams(allStreams, fromTxId, inProgressOk);
}
+ chainAndMakeRedundantStreams(streams, allStreams, fromTxId, inProgressOk);
+ }
+
+ public static void chainAndMakeRedundantStreams(
+ Collection outStreams,
+ PriorityQueue allStreams,
+ long fromTxId, boolean inProgressOk) {
// We want to group together all the streams that start on the same start
// transaction ID. To do this, we maintain an accumulator (acc) of all
// the streams we've seen at a given start transaction ID. When we see a
@@ -246,7 +253,7 @@ public class JournalSet implements JournalManager {
if (accFirstTxId == elis.getFirstTxId()) {
acc.add(elis);
} else if (accFirstTxId < elis.getFirstTxId()) {
- streams.add(new RedundantEditLogInputStream(acc, fromTxId));
+ outStreams.add(new RedundantEditLogInputStream(acc, fromTxId));
acc.clear();
acc.add(elis);
} else if (accFirstTxId > elis.getFirstTxId()) {
@@ -257,7 +264,7 @@ public class JournalSet implements JournalManager {
}
}
if (!acc.isEmpty()) {
- streams.add(new RedundantEditLogInputStream(acc, fromTxId));
+ outStreams.add(new RedundantEditLogInputStream(acc, fromTxId));
acc.clear();
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourcePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourcePolicy.java
index 3896165ff39..03f226c23b7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourcePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourcePolicy.java
@@ -41,6 +41,14 @@ final class NameNodeResourcePolicy {
static boolean areResourcesAvailable(
Collection extends CheckableNameNodeResource> resources,
int minimumRedundantResources) {
+
+ // TODO: workaround:
+ // - during startup, if there are no edits dirs on disk, then there is
+ // a call to areResourcesAvailable() with no dirs at all, which was
+ // previously causing the NN to enter safemode
+ if (resources.isEmpty()) {
+ return true;
+ }
int requiredResourceCount = 0;
int redundantResourceCount = 0;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java
index b6f58877f2f..68fad564a80 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java
@@ -17,18 +17,15 @@
*/
package org.apache.hadoop.hdfs.server.protocol;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.io.Writable;
import com.google.common.base.Function;
import com.google.common.collect.ComparisonChain;
-public class RemoteEditLog implements Writable, Comparable {
+public class RemoteEditLog implements Comparable {
private long startTxId = HdfsConstants.INVALID_TXID;
private long endTxId = HdfsConstants.INVALID_TXID;
+ private boolean isInProgress = false;
public RemoteEditLog() {
}
@@ -36,6 +33,13 @@ public class RemoteEditLog implements Writable, Comparable {
public RemoteEditLog(long startTxId, long endTxId) {
this.startTxId = startTxId;
this.endTxId = endTxId;
+ this.isInProgress = (endTxId == HdfsConstants.INVALID_TXID);
+ }
+
+ public RemoteEditLog(long startTxId, long endTxId, boolean inProgress) {
+ this.startTxId = startTxId;
+ this.endTxId = endTxId;
+ this.isInProgress = inProgress;
}
public long getStartTxId() {
@@ -45,22 +49,18 @@ public class RemoteEditLog implements Writable, Comparable {
public long getEndTxId() {
return endTxId;
}
-
+
+ public boolean isInProgress() {
+ return isInProgress;
+ }
+
@Override
public String toString() {
- return "[" + startTxId + "," + endTxId + "]";
- }
-
- @Override
- public void write(DataOutput out) throws IOException {
- out.writeLong(startTxId);
- out.writeLong(endTxId);
- }
-
- @Override
- public void readFields(DataInput in) throws IOException {
- startTxId = in.readLong();
- endTxId = in.readLong();
+ if (!isInProgress) {
+ return "[" + startTxId + "," + endTxId + "]";
+ } else {
+ return "[" + startTxId + "-? (in-progress)]";
+ }
}
@Override
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
new file mode 100644
index 00000000000..2834631ed96
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
@@ -0,0 +1,186 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+
+option java_package = "org.apache.hadoop.hdfs.qjournal.protocol";
+option java_outer_classname = "QJournalProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "hdfs.proto";
+
+message JournalIdProto {
+ required string identifier = 1;
+}
+
+message RequestInfoProto {
+ required JournalIdProto journalId = 1;
+ required uint64 epoch = 2;
+ required uint64 ipcSerialNumber = 3;
+}
+
+message SegmentStateProto {
+ required uint64 startTxId = 1;
+ required uint64 endTxId = 2;
+ required bool isInProgress = 3;
+ required bytes md5sum = 4;
+}
+
+/**
+ * The storage format used on local disk for previously
+ * accepted decisions.
+ */
+message PersistedRecoveryPaxosData {
+ required SegmentStateProto segmentState = 1;
+ required uint64 acceptedInEpoch = 2;
+}
+
+/**
+ * journal()
+ */
+
+message JournalRequestProto {
+ required RequestInfoProto reqInfo = 1;
+ required uint64 firstTxnId = 2;
+ required uint32 numTxns = 3;
+ required bytes records = 4;
+}
+
+message JournalResponseProto {
+}
+
+/**
+ * startLogSegment()
+ */
+message StartLogSegmentRequestProto {
+ required RequestInfoProto reqInfo = 1;
+ required uint64 txid = 2; // Transaction ID
+}
+
+message StartLogSegmentResponseProto {
+}
+
+/**
+ * finalizeLogSegment()
+ */
+message FinalizeLogSegmentRequestProto {
+ required RequestInfoProto reqInfo = 1;
+ required uint64 startTxId = 2;
+ required uint64 endTxId = 3;
+}
+
+message FinalizeLogSegmentResponseProto {
+}
+
+/**
+ * getJournalState()
+ */
+message GetJournalStateRequestProto {
+ required JournalIdProto jid = 1;
+}
+
+message GetJournalStateResponseProto {
+ required uint64 lastPromisedEpoch = 1;
+ required uint32 httpPort = 2;
+}
+
+/**
+ * newEpoch()
+ */
+message NewEpochRequestProto {
+ required JournalIdProto jid = 1;
+ required NamespaceInfoProto nsInfo = 2;
+ required uint64 epoch = 3;
+}
+
+message NewEpochResponseProto {
+ optional uint64 lastSegmentTxId = 1;
+}
+
+/**
+ * getEditLogManifest()
+ */
+message GetEditLogManifestRequestProto {
+ required JournalIdProto jid = 1;
+ required uint64 sinceTxId = 2; // Transaction ID
+}
+
+message GetEditLogManifestResponseProto {
+ required RemoteEditLogManifestProto manifest = 1;
+ required uint32 httpPort = 2;
+
+ // TODO: we should add nsinfo somewhere
+ // to verify that it matches up with our expectation
+ // required NamespaceInfoProto nsInfo = 2;
+}
+
+/**
+ * prepareRecovery()
+ */
+message PrepareRecoveryRequestProto {
+ required RequestInfoProto reqInfo = 1;
+ required uint64 segmentTxId = 2;
+}
+
+message PrepareRecoveryResponseProto {
+ optional SegmentStateProto segmentState = 1;
+ optional uint64 acceptedInEpoch = 2;
+}
+
+/**
+ * acceptRecovery()
+ */
+message AcceptRecoveryRequestProto {
+ required RequestInfoProto reqInfo = 1;
+
+ /** Details on the segment to recover */
+ required SegmentStateProto stateToAccept = 2;
+
+ /** The URL from which the log may be copied */
+ required string fromURL = 3;
+}
+
+message AcceptRecoveryResponseProto {
+}
+
+
+/**
+ * Protocol used to journal edits to a JournalNode.
+ * See the request and response for details of rpc call.
+ */
+service QJournalProtocolService {
+ rpc getJournalState(GetJournalStateRequestProto) returns (GetJournalStateResponseProto);
+
+ rpc newEpoch(NewEpochRequestProto) returns (NewEpochResponseProto);
+
+ rpc journal(JournalRequestProto) returns (JournalResponseProto);
+
+ rpc startLogSegment(StartLogSegmentRequestProto)
+ returns (StartLogSegmentResponseProto);
+
+ rpc finalizeLogSegment(FinalizeLogSegmentRequestProto)
+ returns (FinalizeLogSegmentResponseProto);
+
+ rpc getEditLogManifest(GetEditLogManifestRequestProto)
+ returns (GetEditLogManifestResponseProto);
+
+ rpc prepareRecovery(PrepareRecoveryRequestProto)
+ returns (PrepareRecoveryResponseProto);
+
+ rpc acceptRecovery(AcceptRecoveryRequestProto)
+ returns (AcceptRecoveryResponseProto);
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
index 4c4bdb5905e..08d25a50c96 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -290,6 +290,7 @@ message BlocksWithLocationsProto {
message RemoteEditLogProto {
required uint64 startTxId = 1; // Starting available edit log transaction
required uint64 endTxId = 2; // Ending available edit log transaction
+ optional bool isInProgress = 3 [default = false];
}
/**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 27f3b9e522f..daead685670 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -240,6 +240,11 @@
+
+ dfs.namenode.edits.journal-plugin.qjournal
+ org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager
+
+
dfs.permissions.enabledtrue
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/index.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/index.html
new file mode 100644
index 00000000000..bc9ea422aad
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/index.html
@@ -0,0 +1,29 @@
+
+
+
+Hadoop Administration
+
+
+
+
+
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/journalstatus.jsp b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/journalstatus.jsp
new file mode 100644
index 00000000000..2c6e5a35213
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/journalstatus.jsp
@@ -0,0 +1,42 @@
+<%
+/*
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.
+ */
+%>
+<%@ page
+ contentType="text/html; charset=UTF-8"
+ import="org.apache.hadoop.hdfs.server.common.JspHelper"
+ import="org.apache.hadoop.util.ServletUtil"
+%>
+<%!
+ //for java.io.Serializable
+ private static final long serialVersionUID = 1L;
+%>
+
+
+
+
+Hadoop JournalNode
+
+
+
JournalNode
+<%= JspHelper.getVersionTable() %>
+
+
+
+Logs
+<%= ServletUtil.htmlFooter() %>
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/proto-journal-web.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/proto-journal-web.xml
new file mode 100644
index 00000000000..b27f82b4a46
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/proto-journal-web.xml
@@ -0,0 +1,17 @@
+
+
+
+@journal.servlet.definitions@
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index b20baa9bd2c..c2212ef4859 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -85,6 +85,7 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.VersionInfo;
+import com.google.common.base.Charsets;
import com.google.common.base.Joiner;
/** Utilities for HDFS tests */
@@ -586,12 +587,21 @@ public class DFSTestUtil {
IOUtils.copyBytes(is, os, s.length(), true);
}
- // Returns url content as string.
+ /**
+ * @return url content as string (UTF-8 encoding assumed)
+ */
public static String urlGet(URL url) throws IOException {
+ return new String(urlGetBytes(url), Charsets.UTF_8);
+ }
+
+ /**
+ * @return URL contents as a byte array
+ */
+ public static byte[] urlGetBytes(URL url) throws IOException {
URLConnection conn = url.openConnection();
ByteArrayOutputStream out = new ByteArrayOutputStream();
IOUtils.copyBytes(conn.getInputStream(), out, 4096, true);
- return out.toString();
+ return out.toByteArray();
}
/**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java
new file mode 100644
index 00000000000..559de47642b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java
@@ -0,0 +1,194 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+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.FileUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.qjournal.server.JournalNode;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+
+public class MiniJournalCluster {
+ public static class Builder {
+ private String baseDir;
+ private int numJournalNodes = 3;
+ private boolean format = true;
+ private Configuration conf;
+
+ public Builder(Configuration conf) {
+ this.conf = conf;
+ }
+
+ public Builder baseDir(String d) {
+ this.baseDir = d;
+ return this;
+ }
+
+ public Builder numJournalNodes(int n) {
+ this.numJournalNodes = n;
+ return this;
+ }
+
+ public Builder format(boolean f) {
+ this.format = f;
+ return this;
+ }
+
+ public MiniJournalCluster build() throws IOException {
+ return new MiniJournalCluster(this);
+ }
+ }
+
+ private static final Log LOG = LogFactory.getLog(MiniJournalCluster.class);
+ private File baseDir;
+ private JournalNode nodes[];
+ private InetSocketAddress ipcAddrs[];
+ private InetSocketAddress httpAddrs[];
+
+ private MiniJournalCluster(Builder b) throws IOException {
+ LOG.info("Starting MiniJournalCluster with " +
+ b.numJournalNodes + " journal nodes");
+
+ if (b.baseDir != null) {
+ this.baseDir = new File(b.baseDir);
+ } else {
+ this.baseDir = new File(MiniDFSCluster.getBaseDirectory());
+ }
+
+ nodes = new JournalNode[b.numJournalNodes];
+ ipcAddrs = new InetSocketAddress[b.numJournalNodes];
+ httpAddrs = new InetSocketAddress[b.numJournalNodes];
+ for (int i = 0; i < b.numJournalNodes; i++) {
+ if (b.format) {
+ File dir = getStorageDir(i);
+ LOG.debug("Fully deleting JN directory " + dir);
+ FileUtil.fullyDelete(dir);
+ }
+ nodes[i] = new JournalNode();
+ nodes[i].setConf(createConfForNode(b, i));
+ nodes[i].start();
+
+ ipcAddrs[i] = nodes[i].getBoundIpcAddress();
+ httpAddrs[i] = nodes[i].getBoundHttpAddress();
+ }
+ }
+
+ /**
+ * Set up the given Configuration object to point to the set of JournalNodes
+ * in this cluster.
+ */
+ public URI getQuorumJournalURI(String jid) {
+ List addrs = Lists.newArrayList();
+ for (InetSocketAddress addr : ipcAddrs) {
+ addrs.add("127.0.0.1:" + addr.getPort());
+ }
+ String addrsVal = Joiner.on(";").join(addrs);
+ LOG.debug("Setting logger addresses to: " + addrsVal);
+ try {
+ return new URI("qjournal://" + addrsVal + "/" + jid);
+ } catch (URISyntaxException e) {
+ throw new AssertionError(e);
+ }
+ }
+
+ /**
+ * Start the JournalNodes in the cluster.
+ */
+ public void start() throws IOException {
+ for (JournalNode jn : nodes) {
+ jn.start();
+ }
+ }
+
+ /**
+ * Shutdown all of the JournalNodes in the cluster.
+ * @throws IOException if one or more nodes failed to stop
+ */
+ public void shutdown() throws IOException {
+ boolean failed = false;
+ for (JournalNode jn : nodes) {
+ try {
+ jn.stopAndJoin(0);
+ } catch (Exception e) {
+ failed = true;
+ LOG.warn("Unable to stop journal node " + jn, e);
+ }
+ }
+ if (failed) {
+ throw new IOException("Unable to shut down. Check log for details");
+ }
+ }
+
+ private Configuration createConfForNode(Builder b, int idx) {
+ Configuration conf = new Configuration(b.conf);
+ File logDir = getStorageDir(idx);
+ conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, logDir.toString());
+ conf.set(DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_KEY, "0.0.0.0:0");
+ conf.set(DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
+ return conf;
+ }
+
+ public File getStorageDir(int idx) {
+ return new File(baseDir, "journalnode-" + idx);
+ }
+
+ public File getCurrentDir(int idx, String jid) {
+ return new File(new File(getStorageDir(idx), jid), "current");
+ }
+
+ public JournalNode getJournalNode(int i) {
+ return nodes[i];
+ }
+
+ public void restartJournalNode(int i) throws InterruptedException, IOException {
+ Configuration conf = new Configuration(nodes[i].getConf());
+ if (nodes[i].isStarted()) {
+ nodes[i].stopAndJoin(0);
+ }
+
+ conf.set(DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_KEY, "127.0.0.1:" +
+ ipcAddrs[i].getPort());
+ conf.set(DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_KEY, "127.0.0.1:" +
+ httpAddrs[i].getPort());
+
+ JournalNode jn = new JournalNode();
+ jn.setConf(conf);
+ jn.start();
+ }
+
+ public int getQuorumSize() {
+ return nodes.length / 2 + 1;
+ }
+
+ public int getNumNodes() {
+ return nodes.length;
+ }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/QJMTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/QJMTestUtil.java
new file mode 100644
index 00000000000..761bdab0a9f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/QJMTestUtil.java
@@ -0,0 +1,41 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.io.DataOutputBuffer;
+
+public abstract class QJMTestUtil {
+
+ public static byte[] createTxnData(int startTxn, int numTxns) throws Exception {
+ DataOutputBuffer buf = new DataOutputBuffer();
+ FSEditLogOp.Writer writer = new FSEditLogOp.Writer(buf);
+
+ for (long txid = startTxn; txid < startTxn + numTxns; txid++) {
+ FSEditLogOp op = NameNodeAdapter.createMkdirOp("tx " + txid);
+ op.setTransactionId(txid);
+ writer.writeOp(op);
+ }
+
+ return Arrays.copyOf(buf.getData(), buf.getLength());
+ }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestMiniJournalCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestMiniJournalCluster.java
new file mode 100644
index 00000000000..8d7cc66accb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestMiniJournalCluster.java
@@ -0,0 +1,51 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.qjournal.server.JournalNode;
+import org.junit.Test;
+
+
+public class TestMiniJournalCluster {
+ @Test
+ public void testStartStop() throws IOException {
+ Configuration conf = new Configuration();
+ MiniJournalCluster c = new MiniJournalCluster.Builder(conf)
+ .build();
+ try {
+ URI uri = c.getQuorumJournalURI("myjournal");
+ String[] addrs = uri.getAuthority().split(";");
+ assertEquals(3, addrs.length);
+
+ JournalNode node = c.getJournalNode(0);
+ String dir = node.getConf().get(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY);
+ assertEquals(MiniDFSCluster.getBaseDirectory() + "journalnode-0",
+ dir);
+ } finally {
+ c.shutdown();
+ }
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java
new file mode 100644
index 00000000000..80c1db277a5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java
@@ -0,0 +1,180 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.ExitUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestNNWithQJM {
+ Configuration conf = new HdfsConfiguration();
+ private MiniJournalCluster mjc;
+ private Path TEST_PATH = new Path("/test-dir");
+ private Path TEST_PATH_2 = new Path("/test-dir");
+
+ @Before
+ public void resetSystemExit() {
+ ExitUtil.resetFirstExitException();
+ }
+
+ @Before
+ public void startJNs() throws Exception {
+ mjc = new MiniJournalCluster.Builder(conf).build();
+ }
+
+ @After
+ public void stopJNs() throws Exception {
+ if (mjc != null) {
+ mjc.shutdown();
+ }
+ }
+
+ @Test
+ public void testLogAndRestart() throws IOException {
+ conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+ MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
+ conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
+ mjc.getQuorumJournalURI("myjournal").toString());
+
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+ .numDataNodes(0)
+ .manageNameDfsDirs(false)
+ .build();
+ try {
+ cluster.getFileSystem().mkdirs(TEST_PATH);
+
+ // Restart the NN and make sure the edit was persisted
+ // and loaded again
+ cluster.restartNameNode();
+
+ assertTrue(cluster.getFileSystem().exists(TEST_PATH));
+ cluster.getFileSystem().mkdirs(TEST_PATH_2);
+
+ // Restart the NN again and make sure both edits are persisted.
+ cluster.restartNameNode();
+ assertTrue(cluster.getFileSystem().exists(TEST_PATH));
+ assertTrue(cluster.getFileSystem().exists(TEST_PATH_2));
+ } finally {
+ cluster.shutdown();
+ }
+ }
+
+ @Test
+ public void testNewNamenodeTakesOverWriter() throws Exception {
+ File nn1Dir = new File(
+ MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image-nn1");
+ File nn2Dir = new File(
+ MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image-nn2");
+
+ conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+ nn1Dir.getAbsolutePath());
+ conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
+ mjc.getQuorumJournalURI("myjournal").toString());
+
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+ .numDataNodes(0)
+ .manageNameDfsDirs(false)
+ .checkExitOnShutdown(false)
+ .build();
+
+ try {
+ cluster.getFileSystem().mkdirs(TEST_PATH);
+
+ // Start a second NN pointed to the same quorum.
+ // We need to copy the image dir from the first NN -- or else
+ // the new NN will just be rejected because of Namespace mismatch.
+ FileUtil.fullyDelete(nn2Dir);
+ FileUtil.copy(nn1Dir, FileSystem.getLocal(conf).getRaw(),
+ new Path(nn2Dir.getAbsolutePath()), false, conf);
+
+ Configuration conf2 = new Configuration();
+ conf2.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+ nn2Dir.getAbsolutePath());
+ conf2.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
+ mjc.getQuorumJournalURI("myjournal").toString());
+ MiniDFSCluster cluster2 = new MiniDFSCluster.Builder(conf2)
+ .numDataNodes(0)
+ .format(false)
+ .manageNameDfsDirs(false)
+ .build();
+
+ // Check that the new cluster sees the edits made on the old cluster
+ try {
+ assertTrue(cluster2.getFileSystem().exists(TEST_PATH));
+ } finally {
+ cluster2.shutdown();
+ }
+
+ // Check that, if we try to write to the old NN
+ // that it aborts.
+ try {
+ cluster.getFileSystem().mkdirs(new Path("/x"));
+ fail("Did not abort trying to write to a fenced NN");
+ } catch (RemoteException re) {
+ GenericTestUtils.assertExceptionContains(
+ "Could not sync enough journals to persistent storage", re);
+ }
+ } finally {
+ //cluster.shutdown();
+ }
+ }
+
+ @Test
+ public void testMismatchedNNIsRejected() throws Exception {
+ conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+ MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
+ conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
+ mjc.getQuorumJournalURI("myjournal").toString());
+
+ // Start a NN, so the storage is formatted with its namespace info.
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+ .numDataNodes(0)
+ .manageNameDfsDirs(false)
+ .build();
+ cluster.shutdown();
+
+ // Create a new (freshly-formatted) NN, which should not be able to
+ // reuse the same journal, since its journal ID would not match.
+ try {
+ cluster = new MiniDFSCluster.Builder(conf)
+ .numDataNodes(0)
+ .manageNameDfsDirs(false)
+ .build();
+ fail("New NN with different namespace should have been rejected");
+ } catch (IOException ioe) {
+ GenericTestUtils.assertExceptionContains(
+ "Unable to start log segment 1: too few journals", ioe);
+ }
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java
new file mode 100644
index 00000000000..265b7603b27
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java
@@ -0,0 +1,130 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.client;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
+import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster.Builder;
+import org.apache.hadoop.hdfs.qjournal.client.AsyncLogger;
+import org.apache.hadoop.hdfs.qjournal.client.AsyncLoggerSet;
+import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+
+
+public class TestEpochsAreUnique {
+ private static final Log LOG = LogFactory.getLog(TestEpochsAreUnique.class);
+ private static final String JID = "testEpochsAreUnique-jid";
+ private static final NamespaceInfo FAKE_NSINFO = new NamespaceInfo(
+ 12345, "mycluster", "my-bp", 0L, 0);
+ private Random r = new Random();
+
+ @Test
+ public void testSingleThreaded() throws IOException {
+ Configuration conf = new Configuration();
+ MiniJournalCluster cluster = new MiniJournalCluster.Builder(conf).build();
+ URI uri = cluster.getQuorumJournalURI(JID);
+ try {
+ // With no failures or contention, epochs should increase one-by-one
+ for (int i = 0; i < 5; i++) {
+ AsyncLoggerSet als = new AsyncLoggerSet(
+ QuorumJournalManager.createLoggers(conf, uri, FAKE_NSINFO));
+ als.createNewUniqueEpoch(FAKE_NSINFO);
+ assertEquals(i + 1, als.getEpoch());
+ }
+
+ long prevEpoch = 5;
+ // With some failures injected, it should still always increase, perhaps
+ // skipping some
+ for (int i = 0; i < 20; i++) {
+ AsyncLoggerSet als = new AsyncLoggerSet(
+ makeFaulty(QuorumJournalManager.createLoggers(conf, uri, FAKE_NSINFO)));
+ long newEpoch = -1;
+ while (true) {
+ try {
+ als.createNewUniqueEpoch(FAKE_NSINFO);
+ newEpoch = als.getEpoch();
+ break;
+ } catch (IOException ioe) {
+ // It's OK to fail to create an epoch, since we randomly inject
+ // faults. It's possible we'll inject faults in too many of the
+ // underlying nodes, and a failure is expected in that case
+ }
+ }
+ LOG.info("Created epoch " + newEpoch);
+ assertTrue("New epoch " + newEpoch + " should be greater than previous " +
+ prevEpoch, newEpoch > prevEpoch);
+ prevEpoch = newEpoch;
+ }
+ } finally {
+ cluster.shutdown();
+ }
+ }
+
+
+ private List makeFaulty(List loggers) {
+ List ret = Lists.newArrayList();
+ for (AsyncLogger l : loggers) {
+ AsyncLogger spy = Mockito.spy(l);
+ Mockito.doAnswer(new SometimesFaulty(0.10f))
+ .when(spy).getJournalState();
+ Mockito.doAnswer(new SometimesFaulty(0.40f))
+ .when(spy).newEpoch(Mockito.anyLong());
+ ret.add(spy);
+ }
+ return ret;
+ }
+
+ private class SometimesFaulty implements Answer> {
+ private float faultProbability;
+
+ public SometimesFaulty(float faultProbability) {
+ this.faultProbability = faultProbability;
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public ListenableFuture answer(InvocationOnMock invocation)
+ throws Throwable {
+ if (r.nextFloat() < faultProbability) {
+ return Futures.immediateFailedFuture(
+ new IOException("Injected fault"));
+ }
+ return (ListenableFuture)invocation.callRealMethod();
+ }
+ }
+
+
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestIPCLoggerChannel.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestIPCLoggerChannel.java
new file mode 100644
index 00000000000..ba893a1b4cb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestIPCLoggerChannel.java
@@ -0,0 +1,130 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.client;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel;
+import org.apache.hadoop.hdfs.qjournal.client.LoggerTooFarBehindException;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
+import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import com.google.common.base.Supplier;
+
+public class TestIPCLoggerChannel {
+ private static final Log LOG = LogFactory.getLog(
+ TestIPCLoggerChannel.class);
+
+ private Configuration conf = new Configuration();
+ private static final NamespaceInfo FAKE_NSINFO = new NamespaceInfo(
+ 12345, "mycluster", "my-bp", 0L, 0);
+ private static final String JID = "test-journalid";
+ private static final InetSocketAddress FAKE_ADDR =
+ new InetSocketAddress(0);
+ private static final byte[] FAKE_DATA = new byte[4096];
+
+ private QJournalProtocol mockProxy = Mockito.mock(QJournalProtocol.class);
+ private IPCLoggerChannel ch;
+
+ private static final int LIMIT_QUEUE_SIZE_MB = 1;
+ private static final int LIMIT_QUEUE_SIZE_BYTES =
+ LIMIT_QUEUE_SIZE_MB * 1024 * 1024;
+
+ @Before
+ public void setupMock() {
+ conf.setInt(DFSConfigKeys.DFS_QJOURNAL_QUEUE_SIZE_LIMIT_KEY,
+ LIMIT_QUEUE_SIZE_MB);
+
+ // Channel to the mock object instead of a real IPC proxy.
+ ch = new IPCLoggerChannel(conf, FAKE_NSINFO, JID, FAKE_ADDR) {
+ @Override
+ protected QJournalProtocol getProxy() throws IOException {
+ return mockProxy;
+ }
+ };
+
+ ch.setEpoch(1);
+ }
+
+ @Test
+ public void testSimpleCall() throws Exception {
+ ch.sendEdits(1, 3, FAKE_DATA).get();
+ Mockito.verify(mockProxy).journal(Mockito.any(),
+ Mockito.eq(1L), Mockito.eq(3), Mockito.same(FAKE_DATA));
+ }
+
+
+ /**
+ * Test that, once the queue eclipses the configure size limit,
+ * calls to journal more data are rejected.
+ */
+ @Test
+ public void testQueueLimiting() throws Exception {
+
+ // Block the underlying fake proxy from actually completing any calls.
+ DelayAnswer delayer = new DelayAnswer(LOG);
+ Mockito.doAnswer(delayer).when(mockProxy).journal(
+ Mockito.any(),
+ Mockito.eq(1L), Mockito.eq(1), Mockito.same(FAKE_DATA));
+
+ // Queue up the maximum number of calls.
+ int numToQueue = LIMIT_QUEUE_SIZE_BYTES / FAKE_DATA.length;
+ for (int i = 1; i <= numToQueue; i++) {
+ ch.sendEdits((long)i, 1, FAKE_DATA);
+ }
+
+ // The accounting should show the correct total number queued.
+ assertEquals(LIMIT_QUEUE_SIZE_BYTES, ch.getQueuedEditsSize());
+
+ // Trying to queue any more should fail.
+ try {
+ ch.sendEdits(numToQueue + 1, 1, FAKE_DATA).get(1, TimeUnit.SECONDS);
+ fail("Did not fail to queue more calls after queue was full");
+ } catch (ExecutionException ee) {
+ if (!(ee.getCause() instanceof LoggerTooFarBehindException)) {
+ throw ee;
+ }
+ }
+
+ delayer.proceed();
+
+ // After we allow it to proceeed, it should chug through the original queue
+ GenericTestUtils.waitFor(new Supplier() {
+ @Override
+ public Boolean get() {
+ return ch.getQueuedEditsSize() == 0;
+ }
+ }, 10, 1000);
+ }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java
new file mode 100644
index 00000000000..a07a06be9fc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java
@@ -0,0 +1,69 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.client;
+
+import static org.junit.Assert.*;
+
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hdfs.qjournal.client.QuorumCall;
+import org.junit.Test;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.util.concurrent.SettableFuture;
+
+public class TestQuorumCall {
+ @Test(timeout=10000)
+ public void testQuorums() throws Exception {
+ Map> futures = ImmutableMap.of(
+ "f1", SettableFuture.create(),
+ "f2", SettableFuture.create(),
+ "f3", SettableFuture.create());
+
+ QuorumCall q = QuorumCall.create(futures);
+ assertEquals(0, q.countResponses());
+
+ futures.get("f1").set("first future");
+ q.waitFor(1, 0, 0, 100000); // wait for 1 response
+ q.waitFor(0, 1, 0, 100000); // wait for 1 success
+ assertEquals(1, q.countResponses());
+
+
+ futures.get("f2").setException(new Exception("error"));
+ assertEquals(2, q.countResponses());
+
+ futures.get("f3").set("second future");
+ q.waitFor(3, 0, 100, 100000); // wait for 3 responses
+ q.waitFor(0, 2, 100, 100000); // 2 successes
+
+ assertEquals(3, q.countResponses());
+ assertEquals("f1=first future,f3=second future",
+ Joiner.on(",").withKeyValueSeparator("=").join(
+ new TreeMap(q.getResults())));
+
+ try {
+ q.waitFor(0, 4, 100, 10);
+ fail("Didn't time out waiting for more responses than came back");
+ } catch (TimeoutException te) {
+ // expected
+ }
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
new file mode 100644
index 00000000000..1beeee52c20
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
@@ -0,0 +1,361 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.client;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.SortedSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
+import org.apache.hadoop.hdfs.qjournal.client.AsyncLogger;
+import org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel;
+import org.apache.hadoop.hdfs.qjournal.client.QuorumException;
+import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
+import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
+import org.apache.hadoop.hdfs.server.namenode.FileJournalManager;
+import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
+import org.apache.hadoop.hdfs.server.namenode.NNStorage;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+/**
+ * Functional tests for QuorumJournalManager.
+ * For true unit tests, see {@link TestQuorumJournalManagerUnit}.
+ */
+public class TestQuorumJournalManager {
+ private static final Log LOG = LogFactory.getLog(
+ TestQuorumJournalManager.class);
+
+ private static final NamespaceInfo FAKE_NSINFO = new NamespaceInfo(
+ 12345, "mycluster", "my-bp", 0L, 0);
+ private static final String JID = "testQuorumJournalManager";
+ private MiniJournalCluster cluster;
+ private Configuration conf;
+ private QuorumJournalManager qjm;
+ private List spies;
+
+ @Before
+ public void setup() throws Exception {
+ conf = new Configuration();
+ cluster = new MiniJournalCluster.Builder(conf)
+ .build();
+
+ qjm = createSpyingQJM();
+ spies = qjm.getLoggerSetForTests().getLoggersForTests();
+
+ qjm.recoverUnfinalizedSegments();
+ assertEquals(1, qjm.getLoggerSetForTests().getEpoch());
+ }
+
+ @After
+ public void shutdown() throws IOException {
+ cluster.shutdown();
+ }
+
+ @Test
+ public void testSingleWriter() throws Exception {
+ writeSegment(qjm, 1, 3, true);
+
+ // Should be finalized
+ checkRecovery(cluster, 1, 3);
+
+ // Start a new segment
+ writeSegment(qjm, 4, 1, true);
+
+ // Should be finalized
+ checkRecovery(cluster, 4, 4);
+ }
+
+ @Test
+ public void testOrchestratedFailures() throws Exception {
+ writeSegment(qjm, 1, 3, true);
+ writeSegment(qjm, 4, 3, true);
+
+ SortedSet serials = Sets.newTreeSet();
+ for (AsyncLogger l : qjm.getLoggerSetForTests().getLoggersForTests()) {
+ IPCLoggerChannel ch = (IPCLoggerChannel)l;
+ ch.waitForAllPendingCalls();
+ serials.add(ch.getNextIpcSerial());
+ }
+
+ // All of the loggers should have sent the same number of RPCs, since there
+ // were no failures.
+ assertEquals(1, serials.size());
+
+ long maxSerial = serials.first();
+ LOG.info("Max IPC serial = " + maxSerial);
+
+ cluster.shutdown();
+
+ cluster = new MiniJournalCluster.Builder(conf)
+ .build();
+ qjm = createSpyingQJM();
+ spies = qjm.getLoggerSetForTests().getLoggersForTests();
+
+ }
+
+ /**
+ * Test case where a new writer picks up from an old one with no failures
+ * and the previous unfinalized segment entirely consistent -- i.e. all
+ * the JournalNodes end at the same transaction ID.
+ */
+ @Test
+ public void testChangeWritersLogsInSync() throws Exception {
+ writeSegment(qjm, 1, 3, false);
+ assertExistsInQuorum(cluster,
+ NNStorage.getInProgressEditsFileName(1));
+
+ // Make a new QJM
+ qjm = new QuorumJournalManager(
+ conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO);
+ qjm.recoverUnfinalizedSegments();
+ checkRecovery(cluster, 1, 3);
+ }
+
+ /**
+ * Test case where a new writer picks up from an old one which crashed
+ * with the three loggers at different txnids
+ */
+ @Test
+ public void testChangeWritersLogsOutOfSync1() throws Exception {
+ // Journal states: [3, 4, 5]
+ // During recovery: [x, 4, 5]
+ // Should recovery to txn 5
+ doOutOfSyncTest(0, 5L);
+ }
+
+ @Test
+ public void testChangeWritersLogsOutOfSync2() throws Exception {
+ // Journal states: [3, 4, 5]
+ // During recovery: [3, x, 5]
+ // Should recovery to txn 5
+ doOutOfSyncTest(1, 5L);
+ }
+
+ @Test
+ public void testChangeWritersLogsOutOfSync3() throws Exception {
+ // Journal states: [3, 4, 5]
+ // During recovery: [3, 4, x]
+ // Should recovery to txn 4
+ doOutOfSyncTest(2, 4L);
+ }
+
+
+ private void doOutOfSyncTest(int missingOnRecoveryIdx,
+ long expectedRecoveryTxnId) throws Exception {
+ EditLogOutputStream stm = qjm.startLogSegment(1);
+
+ failLoggerAtTxn(spies.get(0), 4);
+ failLoggerAtTxn(spies.get(1), 5);
+
+ writeTxns(stm, 1, 3);
+
+ // This should succeed to 2/3 loggers
+ writeTxns(stm, 4, 1);
+
+ // This should only succeed to 1 logger (index 2). Hence it should
+ // fail
+ try {
+ writeTxns(stm, 5, 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);
+ }
+
+ assertExistsInQuorum(cluster,
+ NNStorage.getInProgressEditsFileName(1));
+
+ // Shut down the specified JN, so it's not present during recovery.
+ cluster.getJournalNode(missingOnRecoveryIdx).stopAndJoin(0);
+
+ // Make a new QJM
+ qjm = createSpyingQJM();
+
+ qjm.recoverUnfinalizedSegments();
+ checkRecovery(cluster, 1, expectedRecoveryTxnId);
+ }
+
+
+ private void failLoggerAtTxn(AsyncLogger spy, long txid) {
+ TestQuorumJournalManagerUnit.futureThrows(new IOException("mock failure"))
+ .when(spy).sendEdits(
+ Mockito.eq(txid), Mockito.eq(1), Mockito.any());
+ }
+
+ /**
+ * edit lengths [3,4,5]
+ * first recovery:
+ * - sees [3,4,x]
+ * - picks length 4 for recoveryEndTxId
+ * - calls acceptRecovery()
+ * - crashes before finalizing
+ * second recovery:
+ * - sees [x, 4, 5]
+ * - should pick recovery length 4, even though it saw
+ * a larger txid, because a previous recovery accepted it
+ */
+ @Test
+ public void testRecoverAfterIncompleteRecovery() throws Exception {
+ EditLogOutputStream stm = qjm.startLogSegment(1);
+
+ failLoggerAtTxn(spies.get(0), 4);
+ failLoggerAtTxn(spies.get(1), 5);
+
+ writeTxns(stm, 1, 3);
+
+ // This should succeed to 2/3 loggers
+ writeTxns(stm, 4, 1);
+
+ // This should only succeed to 1 logger (index 2). Hence it should
+ // fail
+ try {
+ writeTxns(stm, 5, 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);
+ }
+
+ // Shut down the logger that has length = 5
+ cluster.getJournalNode(2).stopAndJoin(0);
+
+ qjm = createSpyingQJM();
+ spies = qjm.getLoggerSetForTests().getLoggersForTests();
+
+ // Allow no logger to finalize
+ for (AsyncLogger spy : spies) {
+ TestQuorumJournalManagerUnit.futureThrows(new IOException("injected"))
+ .when(spy).finalizeLogSegment(Mockito.eq(1L),
+ Mockito.eq(4L));
+ }
+ try {
+ qjm.recoverUnfinalizedSegments();
+ fail("Should have failed recovery since no finalization occurred");
+ } catch (IOException ioe) {
+ GenericTestUtils.assertExceptionContains("injected", ioe);
+ }
+
+ // Now bring back the logger that had 5, and run recovery again.
+ // We should recover to 4, even though there's a longer log.
+ cluster.getJournalNode(0).stopAndJoin(0);
+ cluster.restartJournalNode(2);
+
+ qjm = createSpyingQJM();
+ spies = qjm.getLoggerSetForTests().getLoggersForTests();
+ qjm.recoverUnfinalizedSegments();
+ checkRecovery(cluster, 1, 4);
+ }
+
+
+ private QuorumJournalManager createSpyingQJM()
+ throws IOException, URISyntaxException {
+ return new QuorumJournalManager(
+ conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO) {
+ @Override
+ protected List createLoggers() throws IOException {
+ LOG.info("===> make spies");
+ List realLoggers = super.createLoggers();
+ List spies = Lists.newArrayList();
+ for (AsyncLogger logger : realLoggers) {
+ spies.add(Mockito.spy(logger));
+ }
+ return spies;
+ }
+ };
+ }
+
+ private void writeSegment(QuorumJournalManager qjm,
+ int startTxId, int numTxns, boolean finalize) throws IOException {
+ EditLogOutputStream stm = qjm.startLogSegment(startTxId);
+ // Should create in-progress
+ assertExistsInQuorum(cluster,
+ NNStorage.getInProgressEditsFileName(startTxId));
+
+ writeTxns(stm, startTxId, numTxns);
+ if (finalize) {
+ stm.close();
+ qjm.finalizeLogSegment(startTxId, startTxId + numTxns - 1);
+ }
+ }
+
+ private void writeTxns(EditLogOutputStream stm, int startTxId, int numTxns)
+ throws IOException {
+ for (long txid = startTxId; txid < startTxId + numTxns; txid++) {
+ TestQuorumJournalManagerUnit.writeOp(stm, txid);
+ }
+ stm.setReadyToFlush();
+ stm.flush();
+ }
+
+ private void assertExistsInQuorum(MiniJournalCluster cluster,
+ String fname) {
+ int count = 0;
+ for (int i = 0; i < 3; i++) {
+ File dir = cluster.getCurrentDir(i, JID);
+ if (new File(dir, fname).exists()) {
+ count++;
+ }
+ }
+ assertTrue("File " + fname + " should exist in a quorum of dirs",
+ count >= cluster.getQuorumSize());
+ }
+
+ private void checkRecovery(MiniJournalCluster cluster,
+ long segmentTxId, long expectedEndTxId)
+ throws IOException {
+ int numFinalized = 0;
+ for (int i = 0; i < cluster.getNumNodes(); i++) {
+ File logDir = cluster.getCurrentDir(i, JID);
+ EditLogFile elf = FileJournalManager.getLogFile(logDir, segmentTxId);
+ if (elf == null) {
+ continue;
+ }
+ if (!elf.isInProgress()) {
+ numFinalized++;
+ if (elf.getLastTxId() != expectedEndTxId) {
+ fail("File " + elf + " finalized to wrong txid, expected " +
+ expectedEndTxId);
+ }
+ }
+ }
+
+ if (numFinalized < cluster.getQuorumSize()) {
+ fail("Did not find a quorum of finalized logs starting at " +
+ segmentTxId);
+ }
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
new file mode 100644
index 00000000000..e879a729214
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
@@ -0,0 +1,201 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.client;
+
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.eq;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+
+import org.apache.commons.logging.impl.Log4JLogger;
+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.GetJournalStateResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
+import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.log4j.Level;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.stubbing.Stubber;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+
+/**
+ * True unit tests for QuorumJournalManager
+ */
+public class TestQuorumJournalManagerUnit {
+ static {
+ ((Log4JLogger)QuorumJournalManager.LOG).getLogger().setLevel(Level.ALL);
+ }
+ private static final NamespaceInfo FAKE_NSINFO = new NamespaceInfo(
+ 12345, "mycluster", "my-bp", 0L, 0);
+
+ private Configuration conf = new Configuration();
+ private List spyLoggers;
+ private QuorumJournalManager qjm;
+
+ @Before
+ public void setup() throws Exception {
+ spyLoggers = ImmutableList.of(
+ mockLogger(),
+ mockLogger(),
+ mockLogger());
+
+ qjm = new QuorumJournalManager(conf, new URI("qjournal://host/jid"), FAKE_NSINFO) {
+ @Override
+ protected List createLoggers() {
+ return spyLoggers;
+ }
+ };
+
+ for (AsyncLogger logger : spyLoggers) {
+ futureReturns(GetJournalStateResponseProto.newBuilder()
+ .setLastPromisedEpoch(0)
+ .setHttpPort(-1)
+ .build())
+ .when(logger).getJournalState();
+
+ futureReturns(
+ NewEpochResponseProto.newBuilder().build()
+ ).when(logger).newEpoch(Mockito.anyLong());
+ }
+
+ qjm.recoverUnfinalizedSegments();
+ }
+
+ private AsyncLogger mockLogger() {
+ return Mockito.mock(AsyncLogger.class);
+ }
+
+ static Stubber futureReturns(V value) {
+ ListenableFuture ret = Futures.immediateFuture(value);
+ return Mockito.doReturn(ret);
+ }
+
+ static Stubber futureThrows(Throwable t) {
+ ListenableFuture> ret = Futures.immediateFailedFuture(t);
+ return Mockito.doReturn(ret);
+ }
+
+
+ @Test
+ public void testAllLoggersStartOk() throws Exception {
+ futureReturns(null).when(spyLoggers.get(0)).startLogSegment(Mockito.anyLong());
+ futureReturns(null).when(spyLoggers.get(1)).startLogSegment(Mockito.anyLong());
+ futureReturns(null).when(spyLoggers.get(2)).startLogSegment(Mockito.anyLong());
+ qjm.startLogSegment(1);
+ }
+
+ @Test
+ public void testQuorumOfLoggersStartOk() throws Exception {
+ futureReturns(null).when(spyLoggers.get(0)).startLogSegment(Mockito.anyLong());
+ futureReturns(null).when(spyLoggers.get(1)).startLogSegment(Mockito.anyLong());
+ futureThrows(new IOException("logger failed"))
+ .when(spyLoggers.get(2)).startLogSegment(Mockito.anyLong());
+ qjm.startLogSegment(1);
+ }
+
+ @Test
+ public void testQuorumOfLoggersFail() throws Exception {
+ futureReturns(null).when(spyLoggers.get(0)).startLogSegment(Mockito.anyLong());
+ futureThrows(new IOException("logger failed"))
+ .when(spyLoggers.get(1)).startLogSegment(Mockito.anyLong());
+ futureThrows(new IOException("logger failed"))
+ .when(spyLoggers.get(2)).startLogSegment(Mockito.anyLong());
+ try {
+ qjm.startLogSegment(1);
+ fail("Did not throw when quorum failed");
+ } catch (QuorumException qe) {
+ GenericTestUtils.assertExceptionContains("logger failed", qe);
+ }
+ }
+
+ @Test
+ public void testWriteEdits() throws Exception {
+ EditLogOutputStream stm = createLogSegment();
+ writeOp(stm, 1);
+ writeOp(stm, 2);
+
+ stm.setReadyToFlush();
+ writeOp(stm, 3);
+
+ // The flush should log txn 1-2
+ futureReturns(null).when(spyLoggers.get(0)).sendEdits(
+ eq(1L), eq(2), Mockito.any());
+ futureReturns(null).when(spyLoggers.get(1)).sendEdits(
+ eq(1L), eq(2), Mockito.any());
+ futureReturns(null).when(spyLoggers.get(2)).sendEdits(
+ eq(1L), eq(2), Mockito.any());
+ stm.flush();
+
+ // Another flush should now log txn #3
+ stm.setReadyToFlush();
+ futureReturns(null).when(spyLoggers.get(0)).sendEdits(
+ eq(3L), eq(1), Mockito.any());
+ futureReturns(null).when(spyLoggers.get(1)).sendEdits(
+ eq(3L), eq(1), Mockito.any());
+ futureReturns(null).when(spyLoggers.get(2)).sendEdits(
+ eq(3L), eq(1), Mockito.any());
+ stm.flush();
+ }
+
+ @Test
+ public void testWriteEditsOneSlow() throws Exception {
+ EditLogOutputStream stm = createLogSegment();
+ writeOp(stm, 1);
+ stm.setReadyToFlush();
+
+ // Make the first two logs respond immediately
+ futureReturns(null).when(spyLoggers.get(0)).sendEdits(
+ eq(1L), eq(1), Mockito.any());
+ futureReturns(null).when(spyLoggers.get(1)).sendEdits(
+ eq(1L), eq(1), Mockito.any());
+
+ // And the third log not respond
+ SettableFuture slowLog = SettableFuture.create();
+ Mockito.doReturn(slowLog).when(spyLoggers.get(2)).sendEdits(
+ eq(1L), eq(1), Mockito.any());
+ stm.flush();
+ }
+
+ private EditLogOutputStream createLogSegment() throws IOException {
+ futureReturns(null).when(spyLoggers.get(0)).startLogSegment(Mockito.anyLong());
+ futureReturns(null).when(spyLoggers.get(1)).startLogSegment(Mockito.anyLong());
+ futureReturns(null).when(spyLoggers.get(2)).startLogSegment(Mockito.anyLong());
+ EditLogOutputStream stm = qjm.startLogSegment(1);
+ return stm;
+ }
+
+ static void writeOp(EditLogOutputStream stm, long txid) throws IOException {
+ FSEditLogOp op = NameNodeAdapter.createMkdirOp("tx " + txid);
+ op.setTransactionId(txid);
+ stm.write(op);
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
new file mode 100644
index 00000000000..e38630c2762
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
@@ -0,0 +1,166 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.server;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.qjournal.QJMTestUtil;
+import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProtoOrBuilder;
+import org.apache.hadoop.hdfs.qjournal.server.Journal;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestJournal {
+ private static final NamespaceInfo FAKE_NSINFO = new NamespaceInfo(
+ 12345, "mycluster", "my-bp", 0L, 0);
+ private static final NamespaceInfo FAKE_NSINFO_2 = new NamespaceInfo(
+ 6789, "mycluster", "my-bp", 0L, 0);
+
+ private static final String JID = "test-journal";
+
+ private static final File TEST_LOG_DIR = new File(
+ new File(MiniDFSCluster.getBaseDirectory()), "TestJournal");
+
+ private StorageErrorReporter mockErrorReporter = Mockito.mock(
+ StorageErrorReporter.class);
+
+ private Journal journal;
+
+
+ @Before
+ public void setup() throws Exception {
+ FileUtil.fullyDelete(TEST_LOG_DIR);
+ journal = new Journal(TEST_LOG_DIR, mockErrorReporter);
+ }
+
+ @After
+ public void verifyNoStorageErrors() throws Exception{
+ Mockito.verify(mockErrorReporter, Mockito.never())
+ .reportErrorOnFile(Mockito.any());
+ }
+
+ @Test
+ public void testEpochHandling() throws Exception {
+ assertEquals(0, journal.getLastPromisedEpoch());
+ NewEpochResponseProto newEpoch =
+ journal.newEpoch(FAKE_NSINFO, 1);
+ assertFalse(newEpoch.hasLastSegmentTxId());
+ assertEquals(1, journal.getLastPromisedEpoch());
+ journal.newEpoch(FAKE_NSINFO, 3);
+ assertFalse(newEpoch.hasLastSegmentTxId());
+ assertEquals(3, journal.getLastPromisedEpoch());
+ try {
+ journal.newEpoch(FAKE_NSINFO, 3);
+ fail("Should have failed to promise same epoch twice");
+ } catch (IOException ioe) {
+ GenericTestUtils.assertExceptionContains(
+ "Proposed epoch 3 <= last promise 3", ioe);
+ }
+ try {
+ journal.startLogSegment(new RequestInfo(JID, 1L, 1L),
+ 12345L);
+ fail("Should have rejected call from prior epoch");
+ } catch (IOException ioe) {
+ GenericTestUtils.assertExceptionContains(
+ "epoch 1 is less than the last promised epoch 3", ioe);
+ }
+ try {
+ journal.journal(new RequestInfo(JID, 1L, 1L),
+ 100L, 0, new byte[0]);
+ fail("Should have rejected call from prior epoch");
+ } catch (IOException ioe) {
+ GenericTestUtils.assertExceptionContains(
+ "epoch 1 is less than the last promised epoch 3", ioe);
+ }
+ }
+
+ @Test
+ public void testRestartJournal() throws Exception {
+ journal.newEpoch(FAKE_NSINFO, 1);
+ journal.startLogSegment(new RequestInfo("j", 1, 1), 1);
+ journal.journal(new RequestInfo("j", 1, 2), 1, 2,
+ QJMTestUtil.createTxnData(1, 2));
+ // Don't finalize.
+
+ journal.close(); // close to unlock the storage dir
+
+ // Now re-instantiate, make sure history is still there
+ journal = new Journal(TEST_LOG_DIR, mockErrorReporter);
+ assertEquals(1, journal.getLastPromisedEpoch());
+ NewEpochResponseProtoOrBuilder newEpoch = journal.newEpoch(FAKE_NSINFO, 2);
+ assertEquals(1, newEpoch.getLastSegmentTxId());
+ }
+
+ @Test
+ public void testJournalLocking() throws Exception {
+ StorageDirectory sd = journal.getStorage().getStorageDir(0);
+ File lockFile = new File(sd.getRoot(), Storage.STORAGE_FILE_LOCK);
+
+ // Journal should not be locked, since we lazily initialize it.
+ assertFalse(lockFile.exists());
+
+ journal.newEpoch(FAKE_NSINFO, 1);
+ Assume.assumeTrue(journal.getStorage().getStorageDir(0).isLockSupported());
+
+ // Journal should be locked
+ GenericTestUtils.assertExists(lockFile);
+
+ Journal journal2 = new Journal(TEST_LOG_DIR, mockErrorReporter);
+ try {
+ journal2.newEpoch(FAKE_NSINFO, 2);
+ fail("Did not fail to create another journal in same dir");
+ } catch (IOException ioe) {
+ GenericTestUtils.assertExceptionContains(
+ "Cannot lock storage", ioe);
+ }
+
+ journal.close();
+
+ // Journal should no longer be locked after the close() call.
+ journal2.newEpoch(FAKE_NSINFO, 2);
+ }
+
+ @Test
+ public void testNamespaceVerification() throws Exception {
+ journal.newEpoch(FAKE_NSINFO, 1);
+
+ try {
+ journal.newEpoch(FAKE_NSINFO_2, 2);
+ fail("Did not fail newEpoch() when namespaces mismatched");
+ } catch (IOException ioe) {
+ GenericTestUtils.assertExceptionContains(
+ "Incompatible namespaceID", ioe);
+ }
+ }
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
new file mode 100644
index 00000000000..23a819376f1
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
@@ -0,0 +1,241 @@
+/**
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.qjournal.server;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.net.HttpURLConnection;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.qjournal.QJMTestUtil;
+import org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
+import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
+import org.apache.hadoop.hdfs.qjournal.server.Journal;
+import org.apache.hadoop.hdfs.qjournal.server.JournalNode;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+import com.google.common.primitives.Bytes;
+import com.google.common.primitives.Ints;
+
+
+public class TestJournalNode {
+ private static final NamespaceInfo FAKE_NSINFO = new NamespaceInfo(
+ 12345, "mycluster", "my-bp", 0L, 0);
+ private static final String JID = "test-journalid";
+
+ private JournalNode jn;
+ private Journal journal;
+ private Configuration conf = new Configuration();
+ private IPCLoggerChannel ch;
+
+ static {
+ // Avoid an error when we double-initialize JvmMetrics
+ DefaultMetricsSystem.setMiniClusterMode(true);
+ }
+
+ @Before
+ public void setup() throws Exception {
+ conf.set(DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_KEY,
+ "0.0.0.0:0");
+ jn = new JournalNode();
+ jn.setConf(conf);
+ jn.start();
+ journal = jn.getOrCreateJournal(JID);
+ journal.format(FAKE_NSINFO);
+
+ ch = new IPCLoggerChannel(conf, FAKE_NSINFO, JID, jn.getBoundIpcAddress());
+ }
+
+ @After
+ public void teardown() throws Exception {
+ jn.stop(0);
+ }
+
+ @Test
+ public void testJournal() throws Exception {
+ IPCLoggerChannel ch = new IPCLoggerChannel(
+ conf, FAKE_NSINFO, JID, jn.getBoundIpcAddress());
+ ch.newEpoch(1).get();
+ ch.setEpoch(1);
+ ch.startLogSegment(1).get();
+ ch.sendEdits(1, 1, "hello".getBytes(Charsets.UTF_8)).get();
+ }
+
+
+ @Test
+ public void testReturnsSegmentInfoAtEpochTransition() throws Exception {
+ ch.newEpoch(1).get();
+ ch.setEpoch(1);
+ ch.startLogSegment(1).get();
+ ch.sendEdits(1, 2, QJMTestUtil.createTxnData(1, 2)).get();
+
+ // Switch to a new epoch without closing earlier segment
+ NewEpochResponseProto response = ch.newEpoch(2).get();
+ ch.setEpoch(2);
+ assertEquals(1, response.getLastSegmentTxId());
+
+ ch.finalizeLogSegment(1, 2).get();
+
+ // Switch to a new epoch after just closing the earlier segment.
+ response = ch.newEpoch(3).get();
+ ch.setEpoch(3);
+ assertEquals(1, response.getLastSegmentTxId());
+
+ // Start a segment but don't write anything, check newEpoch segment info
+ ch.startLogSegment(3).get();
+ response = ch.newEpoch(4).get();
+ ch.setEpoch(4);
+ assertEquals(3, response.getLastSegmentTxId());
+ }
+
+ @Test
+ public void testHttpServer() throws Exception {
+ InetSocketAddress addr = jn.getBoundHttpAddress();
+ assertTrue(addr.getPort() > 0);
+
+ String urlRoot = "http://localhost:" + addr.getPort();
+
+ // Check default servlets.
+ String pageContents = DFSTestUtil.urlGet(new URL(urlRoot + "/jmx"));
+ assertTrue("Bad contents: " + pageContents,
+ pageContents.contains(
+ "Hadoop:service=JournalNode,name=JvmMetrics"));
+
+ // Check JSP page.
+ pageContents = DFSTestUtil.urlGet(
+ new URL(urlRoot + "/journalstatus.jsp"));
+ assertTrue(pageContents.contains("JournalNode"));
+
+ // Create some edits on server side
+ byte[] EDITS_DATA = QJMTestUtil.createTxnData(1, 3);
+ IPCLoggerChannel ch = new IPCLoggerChannel(
+ conf, FAKE_NSINFO, JID, jn.getBoundIpcAddress());
+ ch.newEpoch(1).get();
+ ch.setEpoch(1);
+ ch.startLogSegment(1).get();
+ ch.sendEdits(1, 3, EDITS_DATA).get();
+ ch.finalizeLogSegment(1, 3).get();
+
+ // Attempt to retrieve via HTTP, ensure we get the data back
+ // including the header we expected
+ byte[] retrievedViaHttp = DFSTestUtil.urlGetBytes(new URL(urlRoot +
+ "/getJournal?segmentTxId=1&jid=" + JID));
+ byte[] expected = Bytes.concat(
+ Ints.toByteArray(HdfsConstants.LAYOUT_VERSION),
+ EDITS_DATA);
+
+ assertArrayEquals(expected, retrievedViaHttp);
+
+ // Attempt to fetch a non-existent file, check that we get an
+ // error status code
+ URL badUrl = new URL(urlRoot + "/getJournal?segmentTxId=12345&jid=" + JID);
+ HttpURLConnection connection = (HttpURLConnection)badUrl.openConnection();
+ try {
+ assertEquals(404, connection.getResponseCode());
+ } finally {
+ connection.disconnect();
+ }
+ }
+
+ /**
+ * Test that the JournalNode performs correctly as a Paxos
+ * Acceptor process.
+ */
+ @Test
+ public void testAcceptRecoveryBehavior() throws Exception {
+ // We need to run newEpoch() first, or else we have no way to distinguish
+ // different proposals for the same decision.
+ try {
+ ch.prepareRecovery(1L).get();
+ fail("Did not throw IllegalState when trying to run paxos without an epoch");
+ } catch (ExecutionException ise) {
+ GenericTestUtils.assertExceptionContains("bad epoch", ise);
+ }
+
+ ch.newEpoch(1).get();
+ ch.setEpoch(1);
+
+ // prepare() with no previously accepted value and no logs present
+ PrepareRecoveryResponseProto prep = ch.prepareRecovery(1L).get();
+ System.err.println("Prep: " + prep);
+ assertFalse(prep.hasAcceptedInEpoch());
+ assertFalse(prep.hasSegmentState());
+
+ // Make a log segment, and prepare again -- this time should see the
+ // segment existing.
+ ch.startLogSegment(1L).get();
+ ch.sendEdits(1L, 1, QJMTestUtil.createTxnData(1, 1)).get();
+
+ prep = ch.prepareRecovery(1L).get();
+ System.err.println("Prep: " + prep);
+ assertFalse(prep.hasAcceptedInEpoch());
+ assertTrue(prep.hasSegmentState());
+
+ // accept() should save the accepted value in persistent storage
+ // TODO: should be able to accept without a URL here
+ ch.acceptRecovery(prep.getSegmentState(), new URL("file:///dev/null")).get();
+
+ // So another prepare() call from a new epoch would return this value
+ ch.newEpoch(2);
+ ch.setEpoch(2);
+ prep = ch.prepareRecovery(1L).get();
+ assertEquals(1L, prep.getAcceptedInEpoch());
+ assertEquals(1L, prep.getSegmentState().getEndTxId());
+
+ // A prepare() or accept() call from an earlier epoch should now be rejected
+ ch.setEpoch(1);
+ try {
+ ch.prepareRecovery(1L).get();
+ fail("prepare from earlier epoch not rejected");
+ } catch (ExecutionException ioe) {
+ GenericTestUtils.assertExceptionContains(
+ "epoch 1 is less than the last promised epoch 2",
+ ioe);
+ }
+ try {
+ ch.acceptRecovery(prep.getSegmentState(), new URL("file:///dev/null")).get();
+ fail("accept from earlier epoch not rejected");
+ } catch (ExecutionException ioe) {
+ GenericTestUtils.assertExceptionContains(
+ "epoch 1 is less than the last promised epoch 2",
+ ioe);
+ }
+ }
+
+ // TODO:
+ // - add test that checks formatting behavior
+ // - add test that checks rejects newEpoch if nsinfo doesn't match
+
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
index 6794591e71c..c3d5aaf6d3f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
@@ -17,11 +17,16 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.spy;
+
import java.io.File;
import java.io.IOException;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -183,6 +188,15 @@ public class NameNodeAdapter {
}
}
+ public static FSEditLogOp createMkdirOp(String path) {
+ MkdirOp op = MkdirOp.getInstance(new FSEditLogOp.OpInstanceCache())
+ .setPath(path)
+ .setTimestamp(0)
+ .setPermissionStatus(new PermissionStatus(
+ "testuser", "testgroup", FsPermission.getDefault()));
+ return op;
+ }
+
/**
* @return the number of blocks marked safe by safemode, or -1
* if safemode is not running.