Browse Source

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.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-3077@1363596 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 12 năm trước cách đây
mục cha
commit
74d4573a23
53 tập tin đã thay đổi với 5697 bổ sung43 xóa
  1. 9 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ExitUtil.java
  2. 4 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-3077.txt
  3. 10 0
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  4. 23 0
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  5. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
  6. 33 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  7. 6 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  8. 114 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java
  9. 272 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java
  10. 330 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
  11. 25 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/LoggerTooFarBehindException.java
  12. 166 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java
  13. 85 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumException.java
  14. 382 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java
  15. 103 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumOutputStream.java
  16. 114 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java
  17. 54 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/RequestInfo.java
  18. 41 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolPB.java
  19. 177 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
  20. 217 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java
  21. 208 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/GetJournalEditServlet.java
  22. 160 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java
  23. 493 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
  24. 202 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
  25. 126 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeHttpServer.java
  26. 160 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java
  27. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
  28. 12 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
  29. 14 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java
  30. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  31. 46 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
  32. 9 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
  33. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourcePolicy.java
  34. 17 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java
  35. 186 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
  36. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
  37. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  38. 29 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/index.html
  39. 42 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/journalstatus.jsp
  40. 17 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/proto-journal-web.xml
  41. 12 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  42. 194 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java
  43. 41 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/QJMTestUtil.java
  44. 51 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestMiniJournalCluster.java
  45. 180 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java
  46. 130 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java
  47. 130 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestIPCLoggerChannel.java
  48. 69 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java
  49. 361 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java
  50. 201 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java
  51. 166 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
  52. 241 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
  53. 14 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java

+ 9 - 0
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

+ 4 - 0
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.

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml

@@ -8,6 +8,9 @@
      <Match>
        <Package name="org.apache.hadoop.hdfs.server.namenode.ha.proto" />
      </Match>
+     <Match>
+       <Package name="org.apache.hadoop.hdfs.qjournal.protocol" />
+     </Match>
      <Match>
        <Bug pattern="EI_EXPOSE_REP" />
      </Match>
@@ -273,4 +276,11 @@
        <Method name="quit" />
        <Bug pattern="DM_EXIT" />
      </Match>
+
+     <!-- More complex cleanup logic confuses findbugs -->
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.qjournal.server.Journal" />
+       <Method name="persistPaxosData" />
+       <Bug pattern="OS_OPEN_STREAM" />
+     </Match>
  </FindBugsFilter>

+ 23 - 0
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -177,6 +177,25 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
               </sources>
             </configuration>
           </execution>
+          <execution>
+            <id>journal</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>compile</goal>
+            </goals>
+            <configuration>
+              <compile>false</compile>
+              <workingDirectory>${project.build.directory}/generated-src/main/jsp</workingDirectory>
+              <webFragmentFile>${project.build.directory}/journal-jsp-servlet-definitions.xml</webFragmentFile>
+              <packageName>org.apache.hadoop.hdfs.server.journalservice</packageName>
+              <sources>
+                <directory>${basedir}/src/main/webapps/journal</directory>
+                <includes>
+                  <include>*.jsp</include>
+                </includes>
+              </sources>
+            </configuration>
+          </execution>
           <execution>
             <id>datanode</id>
             <phase>generate-sources</phase>
@@ -284,6 +303,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                 <loadfile property="hdfs.servlet.definitions" srcFile="${project.build.directory}/hdfs-jsp-servlet-definitions.xml"/>
                 <loadfile property="secondary.servlet.definitions" srcFile="${project.build.directory}/secondary-jsp-servlet-definitions.xml"/>
                 <loadfile property="datanode.servlet.definitions" srcFile="${project.build.directory}/datanode-jsp-servlet-definitions.xml"/>
+                <loadfile property="journal.servlet.definitions" srcFile="${project.build.directory}/journal-jsp-servlet-definitions.xml"/>               
                 <echoproperties destfile="${project.build.directory}/webxml.properties">
                   <propertyset>
                     <propertyref regex=".*.servlet.definitions"/>
@@ -299,6 +319,9 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                 <copy file="${basedir}/src/main/webapps/proto-datanode-web.xml"
                       tofile="${project.build.directory}/webapps/datanode/WEB-INF/web.xml"
                       filtering="true"/>
+                <copy file="${basedir}/src/main/webapps/proto-journal-web.xml"
+                      tofile="${project.build.directory}/webapps/journal/WEB-INF/web.xml"
+                      filtering="true"/>
                 <copy toDir="${project.build.directory}/webapps">
                   <fileset dir="${basedir}/src/main/webapps">
                     <exclude name="**/*.jsp"/>

+ 4 - 0
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"

+ 33 - 0
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;
 }
+

+ 6 - 3
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(

+ 114 - 0
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:
+ * 
+ * <ul>
+ * <li>All methods return {@link ListenableFuture}s instead of synchronous
+ * objects.</li>
+ * <li>The {@link RequestInfo} objects are created by the underlying
+ * implementation.</li>
+ * </ul>
+ */
+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<Void> 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<Void> 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<Void> finalizeLogSegment(
+      long startTxId, long endTxId);
+
+  /**
+   * @return the state of the last epoch on the target node.
+   */
+  public ListenableFuture<GetJournalStateResponseProto> getJournalState();
+
+  /**
+   * Begin a new epoch on the target node.
+   */
+  public ListenableFuture<NewEpochResponseProto> newEpoch(long epoch);
+  
+  /**
+   * Fetch the list of edit logs available on the remote node.
+   */
+  public ListenableFuture<GetEditLogManifestResponseProto> getEditLogManifest(
+      long fromTxnId);
+
+  /**
+   * Prepare recovery. See the HDFS-3077 design document for details.
+   */
+  public ListenableFuture<PrepareRecoveryResponseProto> prepareRecovery(
+      long segmentTxId);
+
+  /**
+   * Accept a recovery proposal. See the HDFS-3077 design document for details.
+   */
+  public ListenableFuture<Void> 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();
+}

+ 272 - 0
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<AsyncLogger> loggers;
+  
+  private static final long INVALID_EPOCH = -1;
+  private long myEpoch = INVALID_EPOCH;
+  
+  public AsyncLoggerSet(List<AsyncLogger> 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<AsyncLogger, NewEpochResponseProto> createNewUniqueEpoch(
+      NamespaceInfo nsInfo) throws IOException {
+    Preconditions.checkState(myEpoch == -1,
+        "epoch already created: epoch=" + myEpoch);
+    
+    Map<AsyncLogger, GetJournalStateResponseProto> 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<AsyncLogger, NewEpochResponseProto> 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
+   */
+  <V> Map<AsyncLogger, V> waitForWriteQuorum(QuorumCall<AsyncLogger, V> 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<AsyncLogger> 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<AsyncLogger, GetJournalStateResponseProto> getJournalState() {
+    Map<AsyncLogger, ListenableFuture<GetJournalStateResponseProto>> calls =
+        Maps.newHashMap();
+    for (AsyncLogger logger : loggers) {
+      calls.put(logger, logger.getJournalState());
+    }
+    return QuorumCall.create(calls);    
+  }
+
+  private QuorumCall<AsyncLogger,NewEpochResponseProto> newEpoch(
+      NamespaceInfo nsInfo,
+      long epoch) {
+    Map<AsyncLogger, ListenableFuture<NewEpochResponseProto>> calls =
+        Maps.newHashMap();
+    for (AsyncLogger logger : loggers) {
+      calls.put(logger, logger.newEpoch(epoch));
+    }
+    return QuorumCall.create(calls);    
+  }
+
+  public QuorumCall<AsyncLogger, Void> startLogSegment(
+      long txid) {
+    Map<AsyncLogger, ListenableFuture<Void>> calls = Maps.newHashMap();
+    for (AsyncLogger logger : loggers) {
+      calls.put(logger, logger.startLogSegment(txid));
+    }
+    return QuorumCall.create(calls);
+  }
+  
+  public QuorumCall<AsyncLogger, Void> finalizeLogSegment(long firstTxId,
+      long lastTxId) {
+    Map<AsyncLogger, ListenableFuture<Void>> calls = Maps.newHashMap();
+    for (AsyncLogger logger : loggers) {
+      calls.put(logger, logger.finalizeLogSegment(firstTxId, lastTxId));
+    }
+    return QuorumCall.create(calls);
+  }
+  
+  public QuorumCall<AsyncLogger, Void> sendEdits(
+      long firstTxnId, int numTxns, byte[] data) {
+    Map<AsyncLogger, ListenableFuture<Void>> calls = Maps.newHashMap();
+    for (AsyncLogger logger : loggers) {
+      ListenableFuture<Void> future = 
+        logger.sendEdits(firstTxnId, numTxns, data);
+      calls.put(logger, future);
+    }
+    return QuorumCall.create(calls);
+  }
+
+  public QuorumCall<AsyncLogger,GetEditLogManifestResponseProto>
+      getEditLogManifest(long fromTxnId) {
+    Map<AsyncLogger,
+        ListenableFuture<GetEditLogManifestResponseProto>> calls
+        = Maps.newHashMap();
+    for (AsyncLogger logger : loggers) {
+      ListenableFuture<GetEditLogManifestResponseProto> future =
+          logger.getEditLogManifest(fromTxnId);
+      calls.put(logger, future);
+    }
+    return QuorumCall.create(calls);
+  }
+
+  QuorumCall<AsyncLogger, PrepareRecoveryResponseProto>
+      prepareRecovery(long segmentTxId) {
+    Map<AsyncLogger,
+      ListenableFuture<PrepareRecoveryResponseProto>> calls
+      = Maps.newHashMap();
+    for (AsyncLogger logger : loggers) {
+      ListenableFuture<PrepareRecoveryResponseProto> future =
+          logger.prepareRecovery(segmentTxId);
+      calls.put(logger, future);
+    }
+    return QuorumCall.create(calls);
+  }
+
+  QuorumCall<AsyncLogger,Void>
+      acceptRecovery(SegmentStateProto log, URL fromURL) {
+    Map<AsyncLogger, ListenableFuture<Void>> calls
+      = Maps.newHashMap();
+    for (AsyncLogger logger : loggers) {
+      ListenableFuture<Void> future =
+          logger.acceptRecovery(log, fromURL);
+      calls.put(logger, future);
+    }
+    return QuorumCall.create(calls);
+  }
+}

+ 330 - 0
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<GetJournalStateResponseProto> getJournalState() {
+    return executor.submit(new Callable<GetJournalStateResponseProto>() {
+      @Override
+      public GetJournalStateResponseProto call() throws IOException {
+        GetJournalStateResponseProto ret =
+            getProxy().getJournalState(journalId);
+        httpPort = ret.getHttpPort();
+        return ret;
+      }
+    });
+  }
+
+  @Override
+  public ListenableFuture<NewEpochResponseProto> newEpoch(
+      final long epoch) {
+    return executor.submit(new Callable<NewEpochResponseProto>() {
+      @Override
+      public NewEpochResponseProto call() throws IOException {
+        return getProxy().newEpoch(journalId, nsInfo, epoch);
+      }
+    });
+  }
+  
+  @Override
+  public ListenableFuture<Void> sendEdits(
+      final long firstTxnId, final int numTxns, final byte[] data) {
+    try {
+      reserveQueueSpace(data.length);
+    } catch (LoggerTooFarBehindException e) {
+      return Futures.immediateFailedFuture(e);
+    }
+    ListenableFuture<Void> ret = null;
+    try {
+      ret = executor.submit(new Callable<Void>() {
+        @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<Void>() {
+          @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<Void> startLogSegment(final long txid) {
+    return executor.submit(new Callable<Void>() {
+      @Override
+      public Void call() throws IOException {
+        getProxy().startLogSegment(createReqInfo(), txid);
+        return null;
+      }
+    });
+  }
+  
+  @Override
+  public ListenableFuture<Void> finalizeLogSegment(
+      final long startTxId, final long endTxId) {
+    return executor.submit(new Callable<Void>() {
+      @Override
+      public Void call() throws IOException {
+        getProxy().finalizeLogSegment(createReqInfo(), startTxId, endTxId);
+        return null;
+      }
+    });
+  }
+  
+  @Override
+  public ListenableFuture<GetEditLogManifestResponseProto> getEditLogManifest(
+      final long fromTxnId) {
+    return executor.submit(new Callable<GetEditLogManifestResponseProto>() {
+      @Override
+      public GetEditLogManifestResponseProto call() throws IOException {
+        return getProxy().getEditLogManifest(journalId, fromTxnId);
+      }
+    });
+  }
+
+  @Override
+  public ListenableFuture<PrepareRecoveryResponseProto> prepareRecovery(
+      final long segmentTxId) {
+    return executor.submit(new Callable<PrepareRecoveryResponseProto>() {
+      @Override
+      public PrepareRecoveryResponseProto call() throws IOException {
+        return getProxy().prepareRecovery(createReqInfo(), segmentTxId);
+      }
+    });
+  }
+
+  @Override
+  public ListenableFuture<Void> acceptRecovery(
+      final SegmentStateProto log, final URL url) {
+    return executor.submit(new Callable<Void>() {
+      @Override
+      public Void call() throws IOException {
+        getProxy().acceptRecovery(createReqInfo(), log, url);
+        return null;
+      }
+    });
+  }
+
+  @Override
+  public String toString() {
+    return "Channel to journal node " + addr; 
+  }
+}

+ 25 - 0
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;
+
+}

+ 166 - 0
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 <KEY> a key used to identify each of the outgoing calls
+ * @param <RESULT> the type of the call result
+ */
+class QuorumCall<KEY, RESULT> {
+  private final Map<KEY, RESULT> successes = Maps.newHashMap();
+  private final Map<KEY, Throwable> exceptions = Maps.newHashMap();
+  
+  static <KEY, RESULT> QuorumCall<KEY, RESULT> create(
+      Map<KEY, ? extends ListenableFuture<RESULT>> calls) {
+    final QuorumCall<KEY, RESULT> qr = new QuorumCall<KEY, RESULT>();
+    for (final Entry<KEY, ? extends ListenableFuture<RESULT>> e : calls.entrySet()) {
+      Preconditions.checkArgument(e.getValue() != null,
+          "null future for key: " + e.getKey());
+      Futures.addCallback(e.getValue(), new FutureCallback<RESULT>() {
+        @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<KEY, RESULT> 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 <K> String mapToString(
+      Map<K, ? extends Message> map) {
+    StringBuilder sb = new StringBuilder();
+    boolean first = true;
+    for (Map.Entry<K, ? extends Message> e : map.entrySet()) {
+      if (!first) {
+        sb.append("\n");
+      }
+      first = false;
+      sb.append(e.getKey()).append(": ")
+        .append(TextFormat.shortDebugString(e.getValue()));
+    }
+    return sb.toString();
+  }
+}

+ 85 - 0
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 <K> the keys for the quorum calls
+   * @param <V> the success response type
+   * @param successes any successful responses returned
+   * @param exceptions the exceptions returned
+   */
+  public static <K, V> QuorumException create(
+      String simpleMsg,
+      Map<K, V> successes,
+      Map<K, Throwable> 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<K, Throwable> 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;
+}

+ 382 - 0
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:
+   * <ul>
+   * <li>This segment will be finalized on a majority
+   * of nodes.</li>
+   * <li>All nodes which contain the finalized segment will
+   * agree on the length.</li>
+   * </ul>
+   * 
+   * @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<AsyncLogger,PrepareRecoveryResponseProto> prepare =
+        loggers.prepareRecovery(segmentTxId);
+    Map<AsyncLogger, PrepareRecoveryResponseProto> 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<AsyncLogger, PrepareRecoveryResponseProto> 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<AsyncLogger,Void> 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<AsyncLogger, Void> finalize =
+        loggers.finalizeLogSegment(logToSync.getStartTxId(), logToSync.getEndTxId()); 
+    loggers.waitForWriteQuorum(finalize, finalizeSegmentTimeoutMs);
+  }
+  
+  private static final Comparator<Entry<AsyncLogger, PrepareRecoveryResponseProto>> RECOVERY_COMPARATOR =
+  new Comparator<Entry<AsyncLogger, PrepareRecoveryResponseProto>>() {
+      @Override
+      public int compare(
+          Entry<AsyncLogger, PrepareRecoveryResponseProto> a,
+          Entry<AsyncLogger, PrepareRecoveryResponseProto> 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<AsyncLogger> createLoggers() throws IOException {
+    return createLoggers(conf, uri, nsInfo);
+  }
+  
+  static List<AsyncLogger> createLoggers(Configuration conf,
+      URI uri, NamespaceInfo nsInfo) throws IOException {
+    List<AsyncLogger> ret = Lists.newArrayList();
+    List<InetSocketAddress> addrs = getLoggerAddresses(uri);
+    String jid = parseJournalId(uri);
+    for (InetSocketAddress addr : addrs) {
+      ret.add(new IPCLoggerChannel(conf, nsInfo, jid, addr));
+    }
+    return ret;
+  }
+ 
+  private static List<InetSocketAddress> 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<InetSocketAddress> 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<AsyncLogger,Void> q = loggers.startLogSegment(txId);
+    loggers.waitForWriteQuorum(q, startSegmentTimeoutMs);
+    return new QuorumOutputStream(loggers);
+  }
+
+  @Override
+  public void finalizeLogSegment(long firstTxId, long lastTxId)
+      throws IOException {
+    QuorumCall<AsyncLogger,Void> 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<AsyncLogger, NewEpochResponseProto> 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<EditLogInputStream> streams,
+      long fromTxnId, boolean inProgressOk) {
+
+    QuorumCall<AsyncLogger,GetEditLogManifestResponseProto> q =
+        loggers.getEditLogManifest(fromTxnId);
+    Map<AsyncLogger, GetEditLogManifestResponseProto> 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<EditLogInputStream> allStreams = 
+        new PriorityQueue<EditLogInputStream>(64,
+            JournalSet.EDIT_LOG_INPUT_STREAM_COMPARATOR);
+    for (Map.Entry<AsyncLogger, GetEditLogManifestResponseProto> 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;
+  }
+
+}

+ 103 - 0
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<AsyncLogger, Void> qcall = loggers.sendEdits(
+          firstTxToFlush, numReadyTxns, data);
+      loggers.waitForWriteQuorum(qcall, 20000); // TODO: configurable timeout
+    }
+  }
+}

+ 114 - 0
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;
+
+}

+ 54 - 0
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;
+  }
+
+}

+ 41 - 0
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 {
+}

+ 177 - 0
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());
+  }
+}

+ 217 - 0
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);
+  }
+}

+ 208 - 0
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:
+ * <ul>
+ * <li>The QuorumJournalManager, when reading edits, fetches the edit streams
+ * from the journal nodes.</li>
+ * <li>During edits synchronization, one journal node will fetch edits from
+ * another journal node.</li>
+ * </ul>
+ */
+@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();
+  }
+}

+ 160 - 0
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();
+  }
+}

+ 493 - 0
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<EditLogFile> 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<File> 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();
+      }
+    }
+  }
+}

+ 202 - 0
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<String, Journal> 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));
+  }
+}

+ 126 - 0
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);
+  }
+}

+ 160 - 0
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);
+  }
+
+}

+ 2 - 1
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 */

+ 12 - 1
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);
   }

+ 14 - 6
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;

+ 1 - 0
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?
     }
   }
 

+ 46 - 11
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<RemoteEditLog> getRemoteEditLogs(long firstTxId) throws IOException {
+  public List<RemoteEditLog> getRemoteEditLogs(long firstTxId) throws IOException {
     File currentDir = sd.getCurrentDir();
     List<EditLogFile> allLogFiles = matchEditLogs(currentDir);
     List<RemoteEditLog> 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<EditLogFile> matchEditLogs(File logDir) throws IOException {
+  public static List<EditLogFile> 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<EditLogFile> getLogFiles(long fromTxId) throws IOException {
+  public List<EditLogFile> getLogFiles(long fromTxId) throws IOException {
     File currentDir = sd.getCurrentDir();
     List<EditLogFile> allLogFiles = matchEditLogs(currentDir);
     List<EditLogFile> 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<EditLogFile> files = matchEditLogs(dir);
+    List<EditLogFile> 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;
     }
     

+ 9 - 2
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<EditLogInputStream> outStreams,
+      PriorityQueue<EditLogInputStream> 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();
     }
   }

+ 8 - 0
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;

+ 17 - 17
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<RemoteEditLog> {
+public class RemoteEditLog implements Comparable<RemoteEditLog> {
   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<RemoteEditLog> {
   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<RemoteEditLog> {
   public long getEndTxId() {
     return endTxId;
   }
-    
-  @Override
-  public String toString() {
-    return "[" + startTxId + "," + endTxId + "]";
-  }
 
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(startTxId);
-    out.writeLong(endTxId);
+  public boolean isInProgress() {
+    return isInProgress;
   }
 
   @Override
-  public void readFields(DataInput in) throws IOException {
-    startTxId = in.readLong();
-    endTxId = in.readLong();
+  public String toString() {
+    if (!isInProgress) {
+      return "[" + startTxId + "," + endTxId + "]";
+    } else {
+      return "[" + startTxId + "-? (in-progress)]";
+    }
   }
   
   @Override

+ 186 - 0
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);
+}

+ 1 - 0
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];
 }
 
 /**

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -240,6 +240,11 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.namenode.edits.journal-plugin.qjournal</name>
+  <value>org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager</value>
+</property>
+
 <property>
   <name>dfs.permissions.enabled</name>
   <value>true</value>

+ 29 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/index.html

@@ -0,0 +1,29 @@
+<meta HTTP-EQUIV="REFRESH" content="0;url=journalstatus.jsp"/>
+<html>
+<!--
+   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.
+-->
+<head><title>Hadoop Administration</title></head>
+
+<body>
+<h1>Hadoop Administration</h1>
+
+<ul> 
+  <li><a href="journalstatus.jsp">Status</a></li> 
+</ul>
+
+</body> 
+</html>

+ 42 - 0
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;
+%>
+
+<!DOCTYPE html>
+<html>
+<link rel="stylesheet" type="text/css" href="/static/hadoop.css">
+<title>Hadoop JournalNode</title>
+    
+<body>
+<h1>JournalNode</h1>
+<%= JspHelper.getVersionTable() %>
+<hr />
+
+<br />
+<b><a href="/logs/">Logs</a></b>
+<%= ServletUtil.htmlFooter() %>

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/proto-journal-web.xml

@@ -0,0 +1,17 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<web-app version="2.4" xmlns="http://java.sun.com/xml/ns/j2ee">
+@journal.servlet.definitions@
+</web-app>

+ 12 - 2
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();
   }
   
   /**

+ 194 - 0
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<String> 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;
+  }
+
+}

+ 41 - 0
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());
+  }
+  
+}

+ 51 - 0
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();
+    }
+  }
+}

+ 180 - 0
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);
+    }
+  }
+}

+ 130 - 0
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<AsyncLogger> makeFaulty(List<AsyncLogger> loggers) {
+    List<AsyncLogger> ret = Lists.newArrayList();
+    for (AsyncLogger l : loggers) {
+      AsyncLogger spy = Mockito.spy(l);
+      Mockito.doAnswer(new SometimesFaulty<Long>(0.10f))
+          .when(spy).getJournalState();
+      Mockito.doAnswer(new SometimesFaulty<Void>(0.40f))
+          .when(spy).newEpoch(Mockito.anyLong());
+      ret.add(spy);
+    }
+    return ret;
+  }
+  
+  private class SometimesFaulty<T> implements Answer<ListenableFuture<T>> {
+    private float faultProbability;
+
+    public SometimesFaulty(float faultProbability) {
+      this.faultProbability = faultProbability;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public ListenableFuture<T> answer(InvocationOnMock invocation)
+        throws Throwable {
+      if (r.nextFloat() < faultProbability) {
+        return Futures.immediateFailedFuture(
+            new IOException("Injected fault"));
+      }
+      return (ListenableFuture<T>)invocation.callRealMethod();
+    }
+  }
+
+
+
+}

+ 130 - 0
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.<RequestInfo>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.<RequestInfo>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<Boolean>() {
+      @Override
+      public Boolean get() {
+        return ch.getQueuedEditsSize() == 0;
+      }
+    }, 10, 1000);
+  }
+
+}

+ 69 - 0
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<String, SettableFuture<String>> futures = ImmutableMap.of(
+        "f1", SettableFuture.<String>create(),
+        "f2", SettableFuture.<String>create(),
+        "f3", SettableFuture.<String>create());
+    
+    QuorumCall<String, String> 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<String, String>(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
+    }
+  }
+}

+ 361 - 0
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<AsyncLogger> 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<Long> 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.<byte[]>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<AsyncLogger> createLoggers() throws IOException {
+            LOG.info("===> make spies");
+            List<AsyncLogger> realLoggers = super.createLoggers();
+            List<AsyncLogger> 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);
+    }
+  }
+}

+ 201 - 0
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<AsyncLogger> 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<AsyncLogger> 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 <V> Stubber futureReturns(V value) {
+    ListenableFuture<V> 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.<byte[]>any());
+    futureReturns(null).when(spyLoggers.get(1)).sendEdits(
+        eq(1L), eq(2), Mockito.<byte[]>any());
+    futureReturns(null).when(spyLoggers.get(2)).sendEdits(
+        eq(1L), eq(2), Mockito.<byte[]>any());
+    stm.flush();
+
+    // Another flush should now log txn #3
+    stm.setReadyToFlush();
+    futureReturns(null).when(spyLoggers.get(0)).sendEdits(
+        eq(3L), eq(1), Mockito.<byte[]>any());
+    futureReturns(null).when(spyLoggers.get(1)).sendEdits(
+        eq(3L), eq(1), Mockito.<byte[]>any());
+    futureReturns(null).when(spyLoggers.get(2)).sendEdits(
+        eq(3L), eq(1), Mockito.<byte[]>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.<byte[]>any());
+    futureReturns(null).when(spyLoggers.get(1)).sendEdits(
+        eq(1L), eq(1), Mockito.<byte[]>any());
+    
+    // And the third log not respond
+    SettableFuture<Void> slowLog = SettableFuture.<Void>create();
+    Mockito.doReturn(slowLog).when(spyLoggers.get(2)).sendEdits(
+        eq(1L), eq(1), Mockito.<byte[]>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);
+  }
+}

+ 166 - 0
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.<File>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);
+    }
+  }
+
+}

+ 241 - 0
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
+   * <em>Acceptor</em> 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
+  
+}

+ 14 - 0
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.