diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh index 9d192501ed2..3e5f20ba20c 100755 --- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh +++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-daemon.sh @@ -141,7 +141,7 @@ case $startStop in echo starting $command, logging to $log cd "$HADOOP_PREFIX" case $command in - namenode|secondarynamenode|datanode|dfs|dfsadmin|fsck|balancer|zkfc) + namenode|secondarynamenode|datanode|journalnode|dfs|dfsadmin|fsck|balancer|zkfc) if [ -z "$HADOOP_HDFS_HOME" ]; then hdfsScript="$HADOOP_PREFIX"/bin/hdfs else diff --git a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml index 131fecfc007..2c324cf3061 100644 --- a/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml +++ b/hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml @@ -239,5 +239,12 @@ group list is separated by a blank. For e.g. "alice,bob users,wheel". A special value of "*" means all users are allowed. + + + security.qjournal.service.protocol.acl + ${HADOOP_HDFS_USER} + ACL for QJournalProtocol, used by the NN to communicate with + JNs when using the QuorumJournalManager for edit logs. + diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java index 2a361d91925..dd0c9556d6e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java @@ -25,6 +25,7 @@ import java.util.Random; import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; import org.apache.commons.logging.Log; @@ -48,6 +49,8 @@ import com.google.common.collect.Sets; */ public abstract class GenericTestUtils { + private static final AtomicInteger sequence = new AtomicInteger(); + /** * Extracts the name of the method where the invocation has happened * @return String name of the invoking method @@ -55,6 +58,14 @@ public abstract class GenericTestUtils { public static String getMethodName() { return Thread.currentThread().getStackTrace()[2].getMethodName(); } + + /** + * Generates a process-wide unique sequence number. + * @return an unique sequence number + */ + public static int uniqueSequenceId() { + return sequence.incrementAndGet(); + } /** * Assert that a given file exists. diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-3077.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-3077.txt new file mode 100644 index 00000000000..46ea04d3baf --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-3077.txt @@ -0,0 +1,88 @@ +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. + +HDFS-3694. Fix getEditLogManifest to fetch httpPort if necessary (todd) + +HDFS-3692. Support purgeEditLogs() call to remotely purge logs on JNs (todd) + +HDFS-3693. JNStorage should read its storage info even before a writer becomes active (todd) + +HDFS-3725. Fix QJM startup when individual JNs have gaps (todd) + +HDFS-3741. Exhaustive failure injection test for skipped RPCs (todd) + +HDFS-3773. TestNNWithQJM fails after HDFS-3741. (atm) + +HDFS-3793. Implement genericized format() in QJM (todd) + +HDFS-3795. QJM: validate journal dir at startup (todd) + +HDFS-3798. Avoid throwing NPE when finalizeSegment() is called on invalid segment (todd) + +HDFS-3799. QJM: handle empty log segments during recovery (todd) + +HDFS-3797. QJM: add segment txid as a parameter to journal() RPC (todd) + +HDFS-3800. improvements to QJM fault testing (todd) + +HDFS-3823. QJM: TestQJMWithFaults fails occasionally because of missed setting of HTTP port. (todd and atm) + +HDFS-3826. QJM: Some trivial logging / exception text improvements. (todd and atm) + +HDFS-3839. QJM: hadoop-daemon.sh should be updated to accept "journalnode" (eli) + +HDFS-3845. Fixes for edge cases in QJM recovery protocol (todd) + +HDFS-3877. QJM: Provide defaults for dfs.journalnode.*address (eli) + +HDFS-3863. Track last "committed" txid in QJM (todd) + +HDFS-3869. Expose non-file journal manager details in web UI (todd) + +HDFS-3884. Journal format() should reset cached values (todd) + +HDFS-3870. Add metrics to JournalNode (todd) + +HDFS-3891. Make selectInputStreams throw IOE instead of RTE (todd) + +HDFS-3726. If a logger misses an RPC, don't retry that logger until next segment (todd) + +HDFS-3893. QJM: Make QJM work with security enabled. (atm) + +HDFS-3897. QJM: TestBlockToken fails after HDFS-3893. (atm) + +HDFS-3898. QJM: enable TCP_NODELAY for IPC (todd) + +HDFS-3885. QJM: optimize log sync when JN is lagging behind (todd) + +HDFS-3900. QJM: avoid validating log segments on log rolls (todd) + +HDFS-3901. QJM: send 'heartbeat' messages to JNs even when they are out-of-sync (todd) + +HDFS-3899. QJM: Add client-side metrics (todd) + +HDFS-3914. QJM: acceptRecovery should abort current segment (todd) + +HDFS-3915. QJM: Failover fails with auth error in secure cluster (todd) + +HDFS-3906. QJM: quorum timeout on failover with large log segment (todd) + +HDFS-3840. JournalNodes log JournalNotFormattedException backtrace error before being formatted (todd) + +HDFS-3894. QJM: testRecoverAfterDoubleFailures can be flaky due to IPC client caching (todd) + +HDFS-3926. QJM: Add user documentation for QJM. (atm) + +HDFS-3943. QJM: remove currently-unused md5sum field (todd) + +HDFS-3950. QJM: misc TODO cleanup, improved log messages, etc. (todd) + +HDFS-3955. QJM: Make acceptRecovery() atomic. (todd) + +HDFS-3956. QJM: purge temporary files when no longer within retention period (todd) + +HDFS-4004. TestJournalNode#testJournal fails because of test case execution order (Chao Shi via todd) + +HDFS-4017. Unclosed FileInputStream in GetJournalEditServlet (Chao Shi via todd) diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml index 7c9f3646e2c..fdaccb99dfb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml @@ -8,6 +8,9 @@ + + + @@ -273,10 +276,18 @@ + + + + + + + + diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml index df7405cada9..c5d45b17112 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml @@ -239,6 +239,25 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> + + journal + generate-sources + + compile + + + false + ${project.build.directory}/generated-src/main/jsp + ${project.build.directory}/journal-jsp-servlet-definitions.xml + org.apache.hadoop.hdfs.server.journalservice + + ${basedir}/src/main/webapps/journal + + *.jsp + + + + datanode generate-sources @@ -349,6 +368,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> + @@ -364,6 +384,9 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogOutputStream.java index 6267871aae0..4cc4b2d960c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogOutputStream.java @@ -84,7 +84,7 @@ class BookKeeperEditLogOutputStream @Override public void close() throws IOException { setReadyToFlush(); - flushAndSync(); + flushAndSync(true); try { lh.close(); } catch (InterruptedException ie) { @@ -130,7 +130,7 @@ class BookKeeperEditLogOutputStream } @Override - public void flushAndSync() throws IOException { + public void flushAndSync(boolean durable) throws IOException { assert(syncLatch != null); try { syncLatch.await(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs index 2b446d1dadc..d7aef66c2ee 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs @@ -30,6 +30,7 @@ function print_usage(){ echo " namenode -format format the DFS filesystem" echo " secondarynamenode run the DFS secondary namenode" echo " namenode run the DFS namenode" + echo " journalnode run the DFS journalnode" echo " zkfc run the ZK Failover Controller daemon" echo " datanode run a DFS datanode" echo " dfsadmin run a DFS admin client" @@ -90,6 +91,9 @@ elif [ "$COMMAND" = "datanode" ] ; then else HADOOP_OPTS="$HADOOP_OPTS -server $HADOOP_DATANODE_OPTS" fi +elif [ "$COMMAND" = "journalnode" ] ; then + CLASS='org.apache.hadoop.hdfs.qjournal.server.JournalNode' + HADOOP_OPTS="$HADOOP_OPTS $HADOOP_JOURNALNODE_OPTS" elif [ "$COMMAND" = "dfs" ] ; then CLASS=org.apache.hadoop.fs.FsShell HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS" diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 441a9056a47..918a790a5a0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -395,4 +395,42 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final String DFS_ENCRYPT_DATA_TRANSFER_KEY = "dfs.encrypt.data.transfer"; public static final boolean DFS_ENCRYPT_DATA_TRANSFER_DEFAULT = false; public static final String DFS_DATA_ENCRYPTION_ALGORITHM_KEY = "dfs.encrypt.data.transfer.algorithm"; + + // 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 String DFS_QJOURNAL_GET_JOURNAL_STATE_TIMEOUT_KEY = "dfs.qjournal.get-journal-state.timeout.ms"; + public static final String DFS_QJOURNAL_NEW_EPOCH_TIMEOUT_KEY = "dfs.qjournal.new-epoch.timeout.ms"; + public static final String DFS_QJOURNAL_WRITE_TXNS_TIMEOUT_KEY = "dfs.qjournal.write-txns.timeout.ms"; + public static final int DFS_QJOURNAL_START_SEGMENT_TIMEOUT_DEFAULT = 20000; + public static final int DFS_QJOURNAL_PREPARE_RECOVERY_TIMEOUT_DEFAULT = 120000; + public static final int DFS_QJOURNAL_ACCEPT_RECOVERY_TIMEOUT_DEFAULT = 120000; + public static final int DFS_QJOURNAL_FINALIZE_SEGMENT_TIMEOUT_DEFAULT = 120000; + public static final int DFS_QJOURNAL_SELECT_INPUT_STREAMS_TIMEOUT_DEFAULT = 20000; + public static final int DFS_QJOURNAL_GET_JOURNAL_STATE_TIMEOUT_DEFAULT = 120000; + public static final int DFS_QJOURNAL_NEW_EPOCH_TIMEOUT_DEFAULT = 120000; + public static final int DFS_QJOURNAL_WRITE_TXNS_TIMEOUT_DEFAULT = 20000; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java index a64a29dc019..27130c32898 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java @@ -75,6 +75,7 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NodeBase; +import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.ToolRunner; @@ -487,6 +488,34 @@ public class DFSUtil { return ret; } + /** + * @return a collection of all configured NN Kerberos principals. + */ + public static Set getAllNnPrincipals(Configuration conf) throws IOException { + Set principals = new HashSet(); + for (String nsId : DFSUtil.getNameServiceIds(conf)) { + if (HAUtil.isHAEnabled(conf, nsId)) { + for (String nnId : DFSUtil.getNameNodeIds(conf, nsId)) { + Configuration confForNn = new Configuration(conf); + NameNode.initializeGenericKeys(confForNn, nsId, nnId); + String principal = SecurityUtil.getServerPrincipal(confForNn + .get(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY), + NameNode.getAddress(confForNn).getHostName()); + principals.add(principal); + } + } else { + Configuration confForNn = new Configuration(conf); + NameNode.initializeGenericKeys(confForNn, nsId, null); + String principal = SecurityUtil.getServerPrincipal(confForNn + .get(DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY), + NameNode.getAddress(confForNn).getHostName()); + principals.add(principal); + } + } + + return principals; + } + /** * Returns list of InetSocketAddress corresponding to HA NN RPC addresses from * the configuration. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java index e8e80a824e2..7268eddb97f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java @@ -23,6 +23,7 @@ import org.apache.hadoop.ha.HAServiceProtocol; import org.apache.hadoop.ha.ZKFCProtocol; import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol; import org.apache.hadoop.hdfs.protocol.ClientProtocol; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol; import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol; import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; @@ -46,6 +47,7 @@ public class HDFSPolicyProvider extends PolicyProvider { new Service("security.inter.datanode.protocol.acl", InterDatanodeProtocol.class), new Service("security.namenode.protocol.acl", NamenodeProtocol.class), + new Service("security.qjournal.service.protocol.acl", QJournalProtocol.class), new Service(CommonConfigurationKeys.SECURITY_HA_SERVICE_PROTOCOL_ACL, HAServiceProtocol.class), new Service(CommonConfigurationKeys.SECURITY_ZKFC_PROTOCOL_ACL, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java index 09032baf398..2d33558ede3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java @@ -326,12 +326,15 @@ public class PBHelper { } public static RemoteEditLogProto convert(RemoteEditLog log) { - return RemoteEditLogProto.newBuilder().setEndTxId(log.getEndTxId()) - .setStartTxId(log.getStartTxId()).build(); + return RemoteEditLogProto.newBuilder() + .setStartTxId(log.getStartTxId()) + .setEndTxId(log.getEndTxId()) + .setIsInProgress(log.isInProgress()).build(); } public static RemoteEditLog convert(RemoteEditLogProto l) { - return new RemoteEditLog(l.getStartTxId(), l.getEndTxId()); + return new RemoteEditLog(l.getStartTxId(), l.getEndTxId(), + l.getIsInProgress()); } public static RemoteEditLogManifestProto convert( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java new file mode 100644 index 00000000000..99b9d6b287d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLogger.java @@ -0,0 +1,154 @@ +/** + * 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.InetSocketAddress; +import java.net.URL; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.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.server.protocol.NamespaceInfo; +import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest; + +import com.google.common.util.concurrent.ListenableFuture; + +/** + * Interface for a remote log which is only communicated with asynchronously. + * This is essentially a wrapper around {@link QJournalProtocol} with the key + * differences being: + * + *
    + *
  • All methods return {@link ListenableFuture}s instead of synchronous + * objects.
  • + *
  • The {@link RequestInfo} objects are created by the underlying + * implementation.
  • + *
+ */ +interface AsyncLogger { + + interface Factory { + AsyncLogger createLogger(Configuration conf, NamespaceInfo nsInfo, + String journalId, InetSocketAddress addr); + } + + /** + * Send a batch of edits to the logger. + * @param segmentTxId the first txid in the current segment + * @param firstTxnId the first txid of the edits. + * @param numTxns the number of transactions in the batch + * @param data the actual data to be sent + */ + public ListenableFuture sendEdits( + final long segmentTxId, final long firstTxnId, + final int numTxns, final byte[] data); + + /** + * Begin writing a new log segment. + * + * @param txid the first txid to be written to the new log + */ + public ListenableFuture startLogSegment(long txid); + + /** + * Finalize a log segment. + * + * @param startTxId the first txid that was written to the segment + * @param endTxId the last txid that was written to the segment + */ + public ListenableFuture finalizeLogSegment( + long startTxId, long endTxId); + + /** + * Allow the remote node to purge edit logs earlier than this. + * @param minTxIdToKeep the min txid which must be retained + */ + public ListenableFuture purgeLogsOlderThan(long minTxIdToKeep); + + /** + * Format the log directory. + * @param nsInfo the namespace info to format with + */ + public ListenableFuture format(NamespaceInfo nsInfo); + + /** + * @return whether or not the remote node has any valid data. + */ + public ListenableFuture isFormatted(); + + /** + * @return the state of the last epoch on the target node. + */ + public ListenableFuture getJournalState(); + + /** + * Begin a new epoch on the target node. + */ + public ListenableFuture newEpoch(long epoch); + + /** + * Fetch the list of edit logs available on the remote node. + */ + public ListenableFuture getEditLogManifest( + long fromTxnId); + + /** + * Prepare recovery. See the HDFS-3077 design document for details. + */ + public ListenableFuture prepareRecovery( + long segmentTxId); + + /** + * Accept a recovery proposal. See the HDFS-3077 design document for details. + */ + public ListenableFuture acceptRecovery(SegmentStateProto log, + URL fromUrl); + + /** + * Set the epoch number used for all future calls. + */ + public void setEpoch(long e); + + /** + * Let the logger know the highest committed txid across all loggers in the + * set. This txid may be higher than the last committed txid for this + * logger. See HDFS-3863 for details. + */ + public void setCommittedTxId(long txid); + + /** + * 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(); + + /** + * Append an HTML-formatted report for this logger's status to the provided + * StringBuilder. This is displayed on the NN web UI. + */ + public void appendHtmlReport(StringBuilder sb); +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java new file mode 100644 index 00000000000..16cd548cb5d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/AsyncLoggerSet.java @@ -0,0 +1,313 @@ +/** + * 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.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 org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest; +import org.apache.jasper.compiler.JspUtil; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Joiner; +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 final List loggers; + + private static final long INVALID_EPOCH = -1; + private long myEpoch = INVALID_EPOCH; + + public AsyncLoggerSet(List loggers) { + this.loggers = ImmutableList.copyOf(loggers); + } + + void setEpoch(long e) { + Preconditions.checkState(!isEpochEstablished(), + "Epoch already established: epoch=%s", myEpoch); + myEpoch = e; + for (AsyncLogger l : loggers) { + l.setEpoch(e); + } + } + + /** + * Set the highest successfully committed txid seen by the writer. + * This should be called after a successful write to a quorum, and is used + * for extra sanity checks against the protocol. See HDFS-3863. + */ + public void setCommittedTxId(long txid) { + for (AsyncLogger logger : loggers) { + logger.setCommittedTxId(txid); + } + } + + /** + * @return true if an epoch has been established. + */ + boolean isEpochEstablished() { + return myEpoch != INVALID_EPOCH; + } + + /** + * @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(); + } + } + + void purgeLogsOlderThan(long minTxIdToKeep) { + for (AsyncLogger logger : loggers) { + logger.purgeLogsOlderThan(minTxIdToKeep); + } + } + + + /** + * 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 + * @param operationName textual description of the operation, for logging + * @return a map of successful results + * @throws QuorumException if a quorum doesn't respond with success + * @throws IOException if the thread is interrupted or times out + */ + Map waitForWriteQuorum(QuorumCall q, + int timeoutMs, String operationName) 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, operationName); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted waiting " + timeoutMs + "ms for a " + + "quorum of nodes to respond."); + } catch (TimeoutException e) { + throw new IOException("Timed out waiting " + timeoutMs + "ms for a " + + "quorum of nodes to respond."); + } + + 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(); + } + + @Override + public String toString() { + return "[" + Joiner.on(", ").join(loggers) + "]"; + } + + /** + * Append an HTML-formatted status readout on the current + * state of the underlying loggers. + * @param sb the StringBuilder to append to + */ + void appendHtmlReport(StringBuilder sb) { + sb.append(""); + sb.append("\n"); + for (AsyncLogger l : loggers) { + sb.append(""); + sb.append(""); + sb.append("\n"); + } + sb.append("
JNStatus
" + JspUtil.escapeXml(l.toString()) + ""); + l.appendHtmlReport(sb); + sb.append("
"); + } + + /** + * @return the (mutable) list of loggers, for use in tests to + * set up spies + */ + @VisibleForTesting + List getLoggersForTests() { + return loggers; + } + + /////////////////////////////////////////////////////////////////////////// + // The rest of this file is simply boilerplate wrappers which fan-out the + // various IPC calls to the underlying AsyncLoggers and wrap the result + // in a QuorumCall. + /////////////////////////////////////////////////////////////////////////// + + public QuorumCall getJournalState() { + Map> calls = + Maps.newHashMap(); + for (AsyncLogger logger : loggers) { + calls.put(logger, logger.getJournalState()); + } + return QuorumCall.create(calls); + } + + public QuorumCall isFormatted() { + Map> calls = Maps.newHashMap(); + for (AsyncLogger logger : loggers) { + calls.put(logger, logger.isFormatted()); + } + return QuorumCall.create(calls); + } + + public QuorumCall newEpoch( + NamespaceInfo nsInfo, + long epoch) { + Map> calls = + Maps.newHashMap(); + for (AsyncLogger logger : loggers) { + calls.put(logger, logger.newEpoch(epoch)); + } + return QuorumCall.create(calls); + } + + public QuorumCall startLogSegment( + long txid) { + Map> calls = Maps.newHashMap(); + for (AsyncLogger logger : loggers) { + calls.put(logger, logger.startLogSegment(txid)); + } + return QuorumCall.create(calls); + } + + public QuorumCall finalizeLogSegment(long firstTxId, + long lastTxId) { + Map> calls = Maps.newHashMap(); + for (AsyncLogger logger : loggers) { + calls.put(logger, logger.finalizeLogSegment(firstTxId, lastTxId)); + } + return QuorumCall.create(calls); + } + + public QuorumCall sendEdits( + long segmentTxId, long firstTxnId, int numTxns, byte[] data) { + Map> calls = Maps.newHashMap(); + for (AsyncLogger logger : loggers) { + ListenableFuture future = + logger.sendEdits(segmentTxId, firstTxnId, numTxns, data); + calls.put(logger, future); + } + return QuorumCall.create(calls); + } + + public QuorumCall + getEditLogManifest(long fromTxnId) { + Map> calls + = Maps.newHashMap(); + for (AsyncLogger logger : loggers) { + ListenableFuture future = + logger.getEditLogManifest(fromTxnId); + calls.put(logger, future); + } + return QuorumCall.create(calls); + } + + QuorumCall + prepareRecovery(long segmentTxId) { + Map> calls + = Maps.newHashMap(); + for (AsyncLogger logger : loggers) { + ListenableFuture future = + logger.prepareRecovery(segmentTxId); + calls.put(logger, future); + } + return QuorumCall.create(calls); + } + + QuorumCall + acceptRecovery(SegmentStateProto log, URL fromURL) { + Map> calls + = Maps.newHashMap(); + for (AsyncLogger logger : loggers) { + ListenableFuture future = + logger.acceptRecovery(log, fromURL); + calls.put(logger, future); + } + return QuorumCall.create(calls); + } + + QuorumCall format(NamespaceInfo nsInfo) { + Map> calls = + Maps.newHashMap(); + for (AsyncLogger logger : loggers) { + ListenableFuture future = + logger.format(nsInfo); + calls.put(logger, future); + } + return QuorumCall.create(calls); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java new file mode 100644 index 00000000000..1c82af858c0 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java @@ -0,0 +1,588 @@ +/** + * 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.security.PrivilegedExceptionAction; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocolPB.PBHelper; +import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.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.hdfs.server.protocol.RemoteEditLogManifest; +import org.apache.hadoop.ipc.ProtobufRpcEngine; +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.security.SecurityUtil; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Stopwatch; +import com.google.common.net.InetAddresses; +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; + protected final InetSocketAddress addr; + private QJournalProtocol proxy; + + private final ListeningExecutorService executor; + private long ipcSerial = 0; + private long epoch = -1; + private long committedTxId = HdfsConstants.INVALID_TXID; + + private final String journalId; + private final NamespaceInfo nsInfo; + private int httpPort = -1; + + private final IPCLoggerChannelMetrics metrics; + + /** + * The number of bytes of edits data still in the queue. + */ + private int queuedEditsSizeBytes = 0; + + /** + * The highest txid that has been successfully logged on the remote JN. + */ + private long highestAckedTxId = 0; + + /** + * Nanotime of the last time we successfully journaled some edits + * to the remote node. + */ + private long lastAckNanos = 0; + + /** + * Nanotime of the last time that committedTxId was update. Used + * to calculate the lag in terms of time, rather than just a number + * of txns. + */ + private long lastCommitNanos = 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; + + /** + * If this logger misses some edits, or restarts in the middle of + * a segment, the writer won't be able to write any more edits until + * the beginning of the next segment. Upon detecting this situation, + * the writer sets this flag to true to avoid sending useless RPCs. + */ + private boolean outOfSync = false; + + /** + * Stopwatch which starts counting on each heartbeat that is sent + */ + private Stopwatch lastHeartbeatStopwatch = new Stopwatch(); + + private static final long HEARTBEAT_INTERVAL_MILLIS = 1000; + + static final Factory FACTORY = new AsyncLogger.Factory() { + @Override + public AsyncLogger createLogger(Configuration conf, NamespaceInfo nsInfo, + String journalId, InetSocketAddress addr) { + return new IPCLoggerChannel(conf, nsInfo, journalId, addr); + } + }; + + 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( + createExecutor()); + + metrics = IPCLoggerChannelMetrics.create(this); + } + + @Override + public synchronized void setEpoch(long epoch) { + this.epoch = epoch; + } + + @Override + public synchronized void setCommittedTxId(long txid) { + Preconditions.checkArgument(txid >= committedTxId, + "Trying to move committed txid backwards in client " + + "old: %s new: %s", committedTxId, txid); + this.committedTxId = txid; + this.lastCommitNanos = System.nanoTime(); + } + + @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; + proxy = createProxy(); + return proxy; + } + + protected QJournalProtocol createProxy() throws IOException { + final Configuration confCopy = new Configuration(conf); + + // Need to set NODELAY or else batches larger than MTU can trigger + // 40ms nagling delays. + confCopy.setBoolean( + CommonConfigurationKeysPublic.IPC_CLIENT_TCPNODELAY_KEY, + true); + + RPC.setProtocolEngine(confCopy, + QJournalProtocolPB.class, ProtobufRpcEngine.class); + return SecurityUtil.doAsLoginUser( + new PrivilegedExceptionAction() { + @Override + public QJournalProtocol run() throws IOException { + RPC.setProtocolEngine(confCopy, + QJournalProtocolPB.class, ProtobufRpcEngine.class); + QJournalProtocolPB pbproxy = RPC.getProxy( + QJournalProtocolPB.class, + RPC.getProtocolVersion(QJournalProtocolPB.class), + addr, confCopy); + return new QJournalProtocolTranslatorPB(pbproxy); + } + }); + } + + + /** + * Separated out for easy overriding in tests. + */ + @VisibleForTesting + protected ExecutorService createExecutor() { + return Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat("Logger channel to " + addr) + .setUncaughtExceptionHandler( + UncaughtExceptionHandlers.systemExit()) + .build()); + } + + @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++, + committedTxId); + } + + @VisibleForTesting + synchronized long getNextIpcSerial() { + return ipcSerial; + } + + public synchronized int getQueuedEditsSize() { + return queuedEditsSizeBytes; + } + + public InetSocketAddress getRemoteAddress() { + return addr; + } + + /** + * @return true if the server has gotten out of sync from the client, + * and thus a log roll is required for this logger to successfully start + * logging more edits. + */ + public synchronized boolean isOutOfSync() { + return outOfSync; + } + + @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 isFormatted() { + return executor.submit(new Callable() { + @Override + public Boolean call() throws IOException { + return getProxy().isFormatted(journalId); + } + }); + } + + @Override + public ListenableFuture getJournalState() { + return executor.submit(new Callable() { + @Override + public GetJournalStateResponseProto call() throws IOException { + GetJournalStateResponseProto ret = + getProxy().getJournalState(journalId); + httpPort = ret.getHttpPort(); + return ret; + } + }); + } + + @Override + public ListenableFuture newEpoch( + final long epoch) { + return executor.submit(new Callable() { + @Override + public NewEpochResponseProto call() throws IOException { + return getProxy().newEpoch(journalId, nsInfo, epoch); + } + }); + } + + @Override + public ListenableFuture sendEdits( + final long segmentTxId, final long firstTxnId, + final int numTxns, final byte[] data) { + try { + reserveQueueSpace(data.length); + } catch (LoggerTooFarBehindException e) { + return Futures.immediateFailedFuture(e); + } + + // When this batch is acked, we use its submission time in order + // to calculate how far we are lagging. + final long submitNanos = System.nanoTime(); + + ListenableFuture ret = null; + try { + ret = executor.submit(new Callable() { + @Override + public Void call() throws IOException { + throwIfOutOfSync(); + + long rpcSendTimeNanos = System.nanoTime(); + try { + getProxy().journal(createReqInfo(), + segmentTxId, firstTxnId, numTxns, data); + } catch (IOException e) { + QuorumJournalManager.LOG.warn( + "Remote journal " + IPCLoggerChannel.this + " failed to " + + "write txns " + firstTxnId + "-" + (firstTxnId + numTxns - 1) + + ". Will try to write to this JN again after the next " + + "log roll.", e); + synchronized (IPCLoggerChannel.this) { + outOfSync = true; + } + throw e; + } finally { + long now = System.nanoTime(); + long rpcTime = TimeUnit.MICROSECONDS.convert( + now - rpcSendTimeNanos, TimeUnit.NANOSECONDS); + long endToEndTime = TimeUnit.MICROSECONDS.convert( + now - submitNanos, TimeUnit.NANOSECONDS); + metrics.addWriteEndToEndLatency(endToEndTime); + metrics.addWriteRpcLatency(rpcTime); + } + synchronized (IPCLoggerChannel.this) { + highestAckedTxId = firstTxnId + numTxns - 1; + lastAckNanos = submitNanos; + } + return null; + } + }); + } finally { + if (ret == null) { + // it didn't successfully get submitted, + // so adjust the queue size back down. + unreserveQueueSpace(data.length); + } else { + // It was submitted to the queue, so adjust the length + // once the call completes, regardless of whether it + // succeeds or fails. + Futures.addCallback(ret, new FutureCallback() { + @Override + public void onFailure(Throwable t) { + unreserveQueueSpace(data.length); + } + + @Override + public void onSuccess(Void t) { + unreserveQueueSpace(data.length); + } + }); + } + } + return ret; + } + + private void throwIfOutOfSync() + throws JournalOutOfSyncException, IOException { + if (isOutOfSync()) { + // Even if we're out of sync, it's useful to send an RPC + // to the remote node in order to update its lag metrics, etc. + heartbeatIfNecessary(); + throw new JournalOutOfSyncException( + "Journal disabled until next roll"); + } + } + + /** + * When we've entered an out-of-sync state, it's still useful to periodically + * send an empty RPC to the server, such that it has the up to date + * committedTxId. This acts as a sanity check during recovery, and also allows + * that node's metrics to be up-to-date about its lag. + * + * In the future, this method may also be used in order to check that the + * current node is still the current writer, even if no edits are being + * written. + */ + private void heartbeatIfNecessary() throws IOException { + if (lastHeartbeatStopwatch.elapsedMillis() > HEARTBEAT_INTERVAL_MILLIS || + !lastHeartbeatStopwatch.isRunning()) { + try { + getProxy().heartbeat(createReqInfo()); + } finally { + // Don't send heartbeats more often than the configured interval, + // even if they fail. + lastHeartbeatStopwatch.reset().start(); + } + } + } + + 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 format(final NamespaceInfo nsInfo) { + return executor.submit(new Callable() { + @Override + public Void call() throws Exception { + getProxy().format(journalId, nsInfo); + return null; + } + }); + } + + @Override + public ListenableFuture startLogSegment(final long txid) { + return executor.submit(new Callable() { + @Override + public Void call() throws IOException { + getProxy().startLogSegment(createReqInfo(), txid); + synchronized (IPCLoggerChannel.this) { + if (outOfSync) { + outOfSync = false; + QuorumJournalManager.LOG.info( + "Restarting previously-stopped writes to " + + IPCLoggerChannel.this + " in segment starting at txid " + + txid); + } + } + return null; + } + }); + } + + @Override + public ListenableFuture finalizeLogSegment( + final long startTxId, final long endTxId) { + return executor.submit(new Callable() { + @Override + public Void call() throws IOException { + throwIfOutOfSync(); + + getProxy().finalizeLogSegment(createReqInfo(), startTxId, endTxId); + return null; + } + }); + } + + @Override + public ListenableFuture purgeLogsOlderThan(final long minTxIdToKeep) { + return executor.submit(new Callable() { + @Override + public Void call() throws Exception { + getProxy().purgeLogsOlderThan(createReqInfo(), minTxIdToKeep); + return null; + } + }); + } + + @Override + public ListenableFuture getEditLogManifest( + final long fromTxnId) { + return executor.submit(new Callable() { + @Override + public RemoteEditLogManifest call() throws IOException { + GetEditLogManifestResponseProto ret = getProxy().getEditLogManifest( + journalId, fromTxnId); + // Update the http port, since we need this to build URLs to any of the + // returned logs. + httpPort = ret.getHttpPort(); + return PBHelper.convert(ret.getManifest()); + } + }); + } + + @Override + public ListenableFuture prepareRecovery( + final long segmentTxId) { + return executor.submit(new Callable() { + @Override + public PrepareRecoveryResponseProto call() throws IOException { + if (httpPort < 0) { + // If the HTTP port hasn't been set yet, force an RPC call so we know + // what the HTTP port should be. + httpPort = getProxy().getJournalState(journalId).getHttpPort(); + } + return getProxy().prepareRecovery(createReqInfo(), segmentTxId); + } + }); + } + + @Override + public ListenableFuture acceptRecovery( + final SegmentStateProto log, final URL url) { + return executor.submit(new Callable() { + @Override + public Void call() throws IOException { + getProxy().acceptRecovery(createReqInfo(), log, url); + return null; + } + }); + } + + @Override + public String toString() { + return InetAddresses.toAddrString(addr.getAddress()) + ':' + + addr.getPort(); + } + + @Override + public synchronized void appendHtmlReport(StringBuilder sb) { + sb.append("Written txid ").append(highestAckedTxId); + long behind = getLagTxns(); + if (behind > 0) { + if (lastAckNanos != 0) { + long lagMillis = getLagTimeMillis(); + sb.append(" (" + behind + " txns/" + lagMillis + "ms behind)"); + } else { + sb.append(" (never written"); + } + } + if (outOfSync) { + sb.append(" (will try to re-sync on next segment)"); + } + } + + public synchronized long getLagTxns() { + return Math.max(committedTxId - highestAckedTxId, 0); + } + + public synchronized long getLagTimeMillis() { + return TimeUnit.MILLISECONDS.convert( + Math.max(lastCommitNanos - lastAckNanos, 0), + TimeUnit.NANOSECONDS); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannelMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannelMetrics.java new file mode 100644 index 00000000000..fde6b998176 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannelMetrics.java @@ -0,0 +1,154 @@ +/** + * 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.InetSocketAddress; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.HdfsConfiguration; +import org.apache.hadoop.metrics2.annotation.Metric; +import org.apache.hadoop.metrics2.annotation.Metrics; +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.metrics2.lib.MetricsRegistry; +import org.apache.hadoop.metrics2.lib.MutableQuantiles; + +import com.google.common.collect.Maps; + +/** + * The metrics for a journal from the writer's perspective. + */ +@Metrics(about="Journal client metrics", context="dfs") +class IPCLoggerChannelMetrics { + final MetricsRegistry registry = new MetricsRegistry("NameNode"); + + private volatile IPCLoggerChannel ch; + + private final MutableQuantiles[] writeEndToEndLatencyQuantiles; + private final MutableQuantiles[] writeRpcLatencyQuantiles; + + + /** + * In the case of the NN transitioning between states, edit logs are closed + * and reopened. Thus, the IPCLoggerChannel instance that writes to a + * given JournalNode may change over the lifetime of the process. + * However, metrics2 doesn't have a function to unregister a set of metrics + * and fails if a new metrics class is registered with the same name + * as the existing one. Hence, we have to maintain our own registry + * ("multiton") here, so that we have exactly one metrics instance + * per JournalNode, and switch out the pointer to the underlying + * IPCLoggerChannel instance. + */ + private static final Map REGISTRY = + Maps.newHashMap(); + + private IPCLoggerChannelMetrics(IPCLoggerChannel ch) { + this.ch = ch; + + Configuration conf = new HdfsConfiguration(); + int[] intervals = + conf.getInts(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY); + if (intervals != null) { + writeEndToEndLatencyQuantiles = new MutableQuantiles[intervals.length]; + writeRpcLatencyQuantiles = new MutableQuantiles[intervals.length]; + for (int i = 0; i < writeEndToEndLatencyQuantiles.length; i++) { + int interval = intervals[i]; + writeEndToEndLatencyQuantiles[i] = registry.newQuantiles( + "writesE2E" + interval + "s", + "End-to-end time for write operations", "ops", "LatencyMicros", interval); + writeRpcLatencyQuantiles[i] = registry.newQuantiles( + "writesRpc" + interval + "s", + "RPC RTT for write operations", "ops", "LatencyMicros", interval); + } + } else { + writeEndToEndLatencyQuantiles = null; + writeRpcLatencyQuantiles = null; + } + } + + private void setChannel(IPCLoggerChannel ch) { + assert ch.getRemoteAddress().equals(this.ch.getRemoteAddress()); + this.ch = ch; + } + + static IPCLoggerChannelMetrics create(IPCLoggerChannel ch) { + String name = getName(ch); + synchronized (REGISTRY) { + IPCLoggerChannelMetrics m = REGISTRY.get(name); + if (m != null) { + m.setChannel(ch); + } else { + m = new IPCLoggerChannelMetrics(ch); + DefaultMetricsSystem.instance().register(name, null, m); + REGISTRY.put(name, m); + } + return m; + } + } + + private static String getName(IPCLoggerChannel ch) { + InetSocketAddress addr = ch.getRemoteAddress(); + String addrStr = addr.getAddress().getHostAddress(); + + // IPv6 addresses have colons, which aren't allowed as part of + // MBean names. Replace with '.' + addrStr = addrStr.replace(':', '.'); + + return "IPCLoggerChannel-" + addrStr + + "-" + addr.getPort(); + } + + @Metric("Is the remote logger out of sync with the quorum") + public String isOutOfSync() { + return Boolean.toString(ch.isOutOfSync()); + } + + @Metric("The number of transactions the remote log is lagging behind the " + + "quorum") + public long getCurrentLagTxns() { + return ch.getLagTxns(); + } + + @Metric("The number of milliseconds the remote log is lagging behind the " + + "quorum") + public long getLagTimeMillis() { + return ch.getLagTimeMillis(); + } + + @Metric("The number of bytes of pending data to be sent to the remote node") + public int getQueuedEditsSize() { + return ch.getQueuedEditsSize(); + } + + public void addWriteEndToEndLatency(long micros) { + if (writeEndToEndLatencyQuantiles != null) { + for (MutableQuantiles q : writeEndToEndLatencyQuantiles) { + q.add(micros); + } + } + } + + public void addWriteRpcLatency(long micros) { + if (writeRpcLatencyQuantiles != null) { + for (MutableQuantiles q : writeRpcLatencyQuantiles) { + q.add(micros); + } + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/LoggerTooFarBehindException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/LoggerTooFarBehindException.java new file mode 100644 index 00000000000..0dbf5bfa5bf --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/LoggerTooFarBehindException.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.qjournal.client; + +import java.io.IOException; + +class LoggerTooFarBehindException extends IOException { + private static final long serialVersionUID = 1L; + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java new file mode 100644 index 00000000000..c42f6b917a6 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java @@ -0,0 +1,230 @@ +/** + * 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.ipc.RemoteException; +import org.apache.hadoop.util.Time; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.protobuf.Message; +import com.google.protobuf.TextFormat; + +/** + * Represents a set of calls for which a quorum of results is needed. + * @param a key used to identify each of the outgoing calls + * @param the type of the call result + */ +class QuorumCall { + private final Map successes = Maps.newHashMap(); + private final Map exceptions = Maps.newHashMap(); + + /** + * Interval, in milliseconds, at which a log message will be made + * while waiting for a quorum call. + */ + private static final int WAIT_PROGRESS_INTERVAL_MILLIS = 1000; + + /** + * Start logging messages at INFO level periodically after waiting for + * this fraction of the configured timeout for any call. + */ + private static final float WAIT_PROGRESS_INFO_THRESHOLD = 0.3f; + /** + * Start logging messages at WARN level after waiting for this + * fraction of the configured timeout for any call. + */ + private static final float WAIT_PROGRESS_WARN_THRESHOLD = 0.7f; + + static QuorumCall create( + Map> calls) { + final QuorumCall qr = new QuorumCall(); + for (final Entry> e : calls.entrySet()) { + Preconditions.checkArgument(e.getValue() != null, + "null future for key: " + e.getKey()); + Futures.addCallback(e.getValue(), new FutureCallback() { + @Override + public void onFailure(Throwable t) { + qr.addException(e.getKey(), t); + } + + @Override + public void onSuccess(RESULT res) { + qr.addResult(e.getKey(), res); + } + }); + } + return qr; + } + + private QuorumCall() { + // Only instantiated from factory method above + } + + /** + * Wait for the quorum to achieve a certain number of responses. + * + * Note that, even after this returns, more responses may arrive, + * causing the return value of other methods in this class to change. + * + * @param minResponses return as soon as this many responses have been + * received, regardless of whether they are successes or exceptions + * @param minSuccesses return as soon as this many successful (non-exception) + * responses have been received + * @param maxExceptions return as soon as this many exception responses + * have been received. Pass 0 to return immediately if any exception is + * received. + * @param millis the number of milliseconds to wait for + * @throws InterruptedException if the thread is interrupted while waiting + * @throws TimeoutException if the specified timeout elapses before + * achieving the desired conditions + */ + public synchronized void waitFor( + int minResponses, int minSuccesses, int maxExceptions, + int millis, String operationName) + throws InterruptedException, TimeoutException { + long st = Time.monotonicNow(); + long nextLogTime = st + (long)(millis * WAIT_PROGRESS_INFO_THRESHOLD); + long et = st + millis; + while (true) { + checkAssertionErrors(); + if (minResponses > 0 && countResponses() >= minResponses) return; + if (minSuccesses > 0 && countSuccesses() >= minSuccesses) return; + if (maxExceptions >= 0 && countExceptions() > maxExceptions) return; + long now = Time.monotonicNow(); + + if (now > nextLogTime) { + long waited = now - st; + String msg = String.format( + "Waited %s ms (timeout=%s ms) for a response for %s", + waited, millis, operationName); + if (waited > millis * WAIT_PROGRESS_WARN_THRESHOLD) { + QuorumJournalManager.LOG.warn(msg); + } else { + QuorumJournalManager.LOG.info(msg); + } + nextLogTime = now + WAIT_PROGRESS_INTERVAL_MILLIS; + } + long rem = et - now; + if (rem <= 0) { + throw new TimeoutException(); + } + rem = Math.min(rem, nextLogTime - now); + rem = Math.max(rem, 1); + wait(rem); + } + } + + /** + * Check if any of the responses came back with an AssertionError. + * If so, it re-throws it, even if there was a quorum of responses. + * This code only runs if assertions are enabled for this class, + * otherwise it should JIT itself away. + * + * This is done since AssertionError indicates programmer confusion + * rather than some kind of expected issue, and thus in the context + * of test cases we'd like to actually fail the test case instead of + * continuing through. + */ + private synchronized void checkAssertionErrors() { + boolean assertsEnabled = false; + assert assertsEnabled = true; // sets to true if enabled + if (assertsEnabled) { + for (Throwable t : exceptions.values()) { + if (t instanceof AssertionError) { + throw (AssertionError)t; + } else if (t instanceof RemoteException && + ((RemoteException)t).getClassName().equals( + AssertionError.class.getName())) { + throw new AssertionError(t); + } + } + } + } + + private synchronized void addResult(KEY k, RESULT res) { + successes.put(k, res); + notifyAll(); + } + + private synchronized void addException(KEY k, Throwable t) { + exceptions.put(k, t); + notifyAll(); + } + + /** + * @return the total number of calls for which a response has been received, + * regardless of whether it threw an exception or returned a successful + * result. + */ + public synchronized int countResponses() { + return successes.size() + exceptions.size(); + } + + /** + * @return the number of calls for which a non-exception response has been + * received. + */ + public synchronized int countSuccesses() { + return successes.size(); + } + + /** + * @return the number of calls for which an exception response has been + * received. + */ + public synchronized int countExceptions() { + return exceptions.size(); + } + + /** + * @return the map of successful responses. A copy is made such that this + * map will not be further mutated, even if further results arrive for the + * quorum. + */ + public synchronized Map getResults() { + return Maps.newHashMap(successes); + } + + public synchronized void rethrowException(String msg) throws QuorumException { + Preconditions.checkState(!exceptions.isEmpty()); + throw QuorumException.create(msg, successes, exceptions); + } + + public static String mapToString( + Map map) { + StringBuilder sb = new StringBuilder(); + boolean first = true; + for (Map.Entry e : map.entrySet()) { + if (!first) { + sb.append("\n"); + } + first = false; + sb.append(e.getKey()).append(": ") + .append(TextFormat.shortDebugString(e.getValue())); + } + return sb.toString(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumException.java new file mode 100644 index 00000000000..446092ebe1c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumException.java @@ -0,0 +1,89 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.qjournal.client; + +import java.io.IOException; +import java.util.Map; + +import org.apache.hadoop.util.StringUtils; + +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; + +/** + * Exception thrown when too many exceptions occur while gathering + * responses to a quorum call. + */ +class QuorumException extends IOException { + + /** + * Create a QuorumException instance with a descriptive message detailing + * the underlying exceptions, as well as any successful responses which + * were returned. + * @param the keys for the quorum calls + * @param the success response type + * @param successes any successful responses returned + * @param exceptions the exceptions returned + */ + public static QuorumException create( + String simpleMsg, + Map successes, + Map exceptions) { + Preconditions.checkArgument(!exceptions.isEmpty(), + "Must pass exceptions"); + + StringBuilder msg = new StringBuilder(); + msg.append(simpleMsg).append(". "); + if (!successes.isEmpty()) { + msg.append(successes.size()).append(" successful responses:\n"); + + Joiner.on("\n") + .useForNull("null [success]") + .withKeyValueSeparator(": ") + .appendTo(msg, successes); + msg.append("\n"); + } + + msg.append(exceptions.size() + " exceptions thrown:\n"); + boolean isFirst = true; + + for (Map.Entry e : exceptions.entrySet()) { + if (!isFirst) { + msg.append("\n"); + } + isFirst = false; + + msg.append(e.getKey()).append(": "); + + if (e.getValue() instanceof RuntimeException) { + msg.append(StringUtils.stringifyException(e.getValue())); + } else if (e.getValue().getLocalizedMessage() != null) { + msg.append(e.getValue().getLocalizedMessage()); + } else { + msg.append(StringUtils.stringifyException(e.getValue())); + } + } + return new QuorumException(msg.toString()); + } + + private QuorumException(String msg) { + super(msg); + } + + private static final long serialVersionUID = 1L; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java new file mode 100644 index 00000000000..6b3503dbba6 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java @@ -0,0 +1,492 @@ +/** + * 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.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.PriorityQueue; +import java.util.concurrent.TimeoutException; + +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.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.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.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.protobuf.TextFormat; + +/** + * 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 int getJournalStateTimeoutMs; + private final int newEpochTimeoutMs; + private final int writeTxnsTimeoutMs; + + // Since these don't occur during normal operation, we can + // use rather lengthy timeouts, and don't need to make them + // configurable. + private static final int FORMAT_TIMEOUT_MS = 60000; + private static final int HASDATA_TIMEOUT_MS = 60000; + + private final Configuration conf; + private final URI uri; + private final NamespaceInfo nsInfo; + private boolean isActiveWriter; + + private final AsyncLoggerSet loggers; + + private int outputBufferCapacity = 512 * 1024; + + public QuorumJournalManager(Configuration conf, + URI uri, NamespaceInfo nsInfo) throws IOException { + this(conf, uri, nsInfo, IPCLoggerChannel.FACTORY); + } + + QuorumJournalManager(Configuration conf, + URI uri, NamespaceInfo nsInfo, + AsyncLogger.Factory loggerFactory) throws IOException { + Preconditions.checkArgument(conf != null, "must be configured"); + + this.conf = conf; + this.uri = uri; + this.nsInfo = nsInfo; + this.loggers = new AsyncLoggerSet(createLoggers(loggerFactory)); + + // 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); + this.getJournalStateTimeoutMs = conf.getInt( + DFSConfigKeys.DFS_QJOURNAL_GET_JOURNAL_STATE_TIMEOUT_KEY, + DFSConfigKeys.DFS_QJOURNAL_GET_JOURNAL_STATE_TIMEOUT_DEFAULT); + this.newEpochTimeoutMs = conf.getInt( + DFSConfigKeys.DFS_QJOURNAL_NEW_EPOCH_TIMEOUT_KEY, + DFSConfigKeys.DFS_QJOURNAL_NEW_EPOCH_TIMEOUT_DEFAULT); + this.writeTxnsTimeoutMs = conf.getInt( + DFSConfigKeys.DFS_QJOURNAL_WRITE_TXNS_TIMEOUT_KEY, + DFSConfigKeys.DFS_QJOURNAL_WRITE_TXNS_TIMEOUT_DEFAULT); + } + + protected List createLoggers( + AsyncLogger.Factory factory) throws IOException { + return createLoggers(conf, uri, nsInfo, factory); + } + + 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); + } + + + /** + * Fence any previous writers, and obtain a unique epoch number + * for write-access to the journal nodes. + * + * @return the new, unique epoch number + */ + Map createNewUniqueEpoch() + throws IOException { + Preconditions.checkState(!loggers.isEpochEstablished(), + "epoch already created"); + + Map lastPromises = + loggers.waitForWriteQuorum(loggers.getJournalState(), + getJournalStateTimeoutMs, "getJournalState()"); + + long maxPromised = Long.MIN_VALUE; + for (GetJournalStateResponseProto resp : lastPromises.values()) { + maxPromised = Math.max(maxPromised, resp.getLastPromisedEpoch()); + } + assert maxPromised >= 0; + + long myEpoch = maxPromised + 1; + Map resps = + loggers.waitForWriteQuorum(loggers.newEpoch(nsInfo, myEpoch), + newEpochTimeoutMs, "newEpoch(" + myEpoch + ")"); + + loggers.setEpoch(myEpoch); + return resps; + } + + @Override + public void format(NamespaceInfo nsInfo) throws IOException { + QuorumCall call = loggers.format(nsInfo); + try { + call.waitFor(loggers.size(), loggers.size(), 0, FORMAT_TIMEOUT_MS, + "format"); + } catch (InterruptedException e) { + throw new IOException("Interrupted waiting for format() response"); + } catch (TimeoutException e) { + throw new IOException("Timed out waiting for format() response"); + } + + if (call.countExceptions() > 0) { + call.rethrowException("Could not format one or more JournalNodes"); + } + } + + @Override + public boolean hasSomeData() throws IOException { + QuorumCall call = + loggers.isFormatted(); + + try { + call.waitFor(loggers.size(), 0, 0, HASDATA_TIMEOUT_MS, "hasSomeData"); + } catch (InterruptedException e) { + throw new IOException("Interrupted while determining if JNs have data"); + } catch (TimeoutException e) { + throw new IOException("Timed out waiting for response from loggers"); + } + + if (call.countExceptions() > 0) { + call.rethrowException( + "Unable to check if JNs are ready for formatting"); + } + + // If any of the loggers returned with a non-empty manifest, then + // we should prompt for format. + for (Boolean hasData : call.getResults().values()) { + if (hasData) { + return true; + } + } + + // Otherwise, none were formatted, we can safely format. + return false; + } + + /** + * Run recovery/synchronization for a specific segment. + * Postconditions: + *
    + *
  • This segment will be finalized on a majority + * of nodes.
  • + *
  • All nodes which contain the finalized segment will + * agree on the length.
  • + *
+ * + * @param segmentTxId the starting txid of the segment + * @throws IOException + */ + private void recoverUnclosedSegment(long segmentTxId) throws IOException { + Preconditions.checkArgument(segmentTxId > 0); + LOG.info("Beginning recovery of unclosed segment starting at txid " + + segmentTxId); + + // Step 1. Prepare recovery + QuorumCall prepare = + loggers.prepareRecovery(segmentTxId); + Map prepareResponses= + loggers.waitForWriteQuorum(prepare, prepareRecoveryTimeoutMs, + "prepareRecovery(" + segmentTxId + ")"); + LOG.info("Recovery prepare phase complete. Responses:\n" + + QuorumCall.mapToString(prepareResponses)); + + // Determine the logger who either: + // a) Has already accepted a previous proposal that's higher than any + // other + // + // OR, if no such logger exists: + // + // b) Has the longest log starting at this transaction ID + + // TODO: we should collect any "ties" and pass the URL for all of them + // when syncing, so we can tolerate failure during recovery better. + Entry bestEntry = Collections.max( + prepareResponses.entrySet(), SegmentRecoveryComparator.INSTANCE); + 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 { + // None of the responses to prepareRecovery() had a segment at the given + // txid. This can happen for example in the following situation: + // - 3 JNs: JN1, JN2, JN3 + // - writer starts segment 101 on JN1, then crashes before + // writing to JN2 and JN3 + // - during newEpoch(), we saw the segment on JN1 and decide to + // recover segment 101 + // - before prepare(), JN1 crashes, and we only talk to JN2 and JN3, + // neither of which has any entry for this log. + // In this case, it is allowed to do nothing for recovery, since the + // segment wasn't started on a quorum of nodes. + + // Sanity check: we should only get here if none of the responses had + // a log. This should be a postcondition of the recovery comparator, + // but a bug in the comparator might cause us to get here. + for (PrepareRecoveryResponseProto resp : prepareResponses.values()) { + assert !resp.hasSegmentState() : + "One of the loggers had a response, but no best logger " + + "was found."; + } + + LOG.info("None of the responders had a log to recover: " + + QuorumCall.mapToString(prepareResponses)); + return; + } + + SegmentStateProto logToSync = bestResponse.getSegmentState(); + assert segmentTxId == logToSync.getStartTxId(); + + // Sanity check: none of the loggers should be aware of a higher + // txid than the txid we intend to truncate to + for (Map.Entry e : + prepareResponses.entrySet()) { + AsyncLogger logger = e.getKey(); + PrepareRecoveryResponseProto resp = e.getValue(); + + if (resp.hasLastCommittedTxId() && + resp.getLastCommittedTxId() > logToSync.getEndTxId()) { + throw new AssertionError("Decided to synchronize log to " + logToSync + + " but logger " + logger + " had seen txid " + + resp.getLastCommittedTxId() + " committed"); + } + } + + URL syncFromUrl = bestLogger.buildURLToFetchLogs(segmentTxId); + + QuorumCall accept = loggers.acceptRecovery(logToSync, syncFromUrl); + loggers.waitForWriteQuorum(accept, acceptRecoveryTimeoutMs, + "acceptRecovery(" + TextFormat.shortDebugString(logToSync) + ")"); + + // If one of the loggers above missed the synchronization step above, but + // we send a finalize() here, that's OK. It validates the log before + // finalizing. Hence, even if it is not "in sync", it won't incorrectly + // finalize. + QuorumCall finalize = + loggers.finalizeLogSegment(logToSync.getStartTxId(), logToSync.getEndTxId()); + loggers.waitForWriteQuorum(finalize, finalizeSegmentTimeoutMs, + String.format("finalizeLogSegment(%s-%s)", + logToSync.getStartTxId(), + logToSync.getEndTxId())); + } + + static List createLoggers(Configuration conf, + URI uri, NamespaceInfo nsInfo, AsyncLogger.Factory factory) + throws IOException { + List ret = Lists.newArrayList(); + List addrs = getLoggerAddresses(uri); + String jid = parseJournalId(uri); + for (InetSocketAddress addr : addrs) { + ret.add(factory.createLogger(conf, nsInfo, jid, addr)); + } + return ret; + } + + private static List getLoggerAddresses(URI uri) + throws IOException { + String authority = uri.getAuthority(); + Preconditions.checkArgument(authority != null && !authority.isEmpty(), + "URI has no authority: " + uri); + + String[] parts = StringUtils.split(authority, ';'); + for (int i = 0; i < parts.length; i++) { + parts[i] = parts[i].trim(); + } + + if (parts.length % 2 == 0) { + LOG.warn("Quorum journal URI '" + uri + "' has an even number " + + "of Journal Nodes specified. This is not recommended!"); + } + + List addrs = Lists.newArrayList(); + for (String addr : parts) { + addrs.add(NetUtils.createSocketAddr( + addr, DFSConfigKeys.DFS_JOURNALNODE_RPC_PORT_DEFAULT)); + } + return addrs; + } + + @Override + public EditLogOutputStream startLogSegment(long txId) throws IOException { + Preconditions.checkState(isActiveWriter, + "must recover segments before starting a new one"); + QuorumCall q = loggers.startLogSegment(txId); + loggers.waitForWriteQuorum(q, startSegmentTimeoutMs, + "startLogSegment(" + txId + ")"); + return new QuorumOutputStream(loggers, txId, + outputBufferCapacity, writeTxnsTimeoutMs); + } + + @Override + public void finalizeLogSegment(long firstTxId, long lastTxId) + throws IOException { + QuorumCall q = loggers.finalizeLogSegment( + firstTxId, lastTxId); + loggers.waitForWriteQuorum(q, finalizeSegmentTimeoutMs, + String.format("finalizeLogSegment(%s-%s)", firstTxId, lastTxId)); + } + + @Override + public void setOutputBufferCapacity(int size) { + outputBufferCapacity = size; + } + + @Override + public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException { + // This purges asynchronously -- there's no need to wait for a quorum + // here, because it's always OK to fail. + LOG.info("Purging remote journals older than txid " + minTxIdToKeep); + loggers.purgeLogsOlderThan(minTxIdToKeep); + } + + @Override + public void recoverUnfinalizedSegments() throws IOException { + Preconditions.checkState(!isActiveWriter, "already active writer"); + + LOG.info("Starting recovery process for unclosed journal segments..."); + Map resps = createNewUniqueEpoch(); + LOG.info("Successfully started new epoch " + loggers.getEpoch()); + + if (LOG.isDebugEnabled()) { + LOG.debug("newEpoch(" + loggers.getEpoch() + ") responses:\n" + + QuorumCall.mapToString(resps)); + } + + long mostRecentSegmentTxId = Long.MIN_VALUE; + for (NewEpochResponseProto r : resps.values()) { + if (r.hasLastSegmentTxId()) { + mostRecentSegmentTxId = Math.max(mostRecentSegmentTxId, + r.getLastSegmentTxId()); + } + } + + // On a completely fresh system, none of the journals have any + // segments, so there's nothing to recover. + if (mostRecentSegmentTxId != Long.MIN_VALUE) { + recoverUnclosedSegment(mostRecentSegmentTxId); + } + isActiveWriter = true; + } + + @Override + public void close() throws IOException { + loggers.close(); + } + + @Override + public void selectInputStreams(Collection streams, + long fromTxnId, boolean inProgressOk) throws IOException { + + QuorumCall q = + loggers.getEditLogManifest(fromTxnId); + Map resps = + loggers.waitForWriteQuorum(q, selectInputStreamsTimeoutMs, + "selectInputStreams"); + + LOG.debug("selectInputStream manifests:\n" + + Joiner.on("\n").withKeyValueSeparator(": ").join(resps)); + + final PriorityQueue allStreams = + new PriorityQueue(64, + JournalSet.EDIT_LOG_INPUT_STREAM_COMPARATOR); + for (Map.Entry e : resps.entrySet()) { + AsyncLogger logger = e.getKey(); + RemoteEditLogManifest manifest = e.getValue(); + + for (RemoteEditLog remoteLog : manifest.getLogs()) { + URL url = logger.buildURLToFetchLogs(remoteLog.getStartTxId()); + + 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 "QJM to " + loggers; + } + + @VisibleForTesting + AsyncLoggerSet getLoggerSetForTests() { + return loggers; + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumOutputStream.java new file mode 100644 index 00000000000..7a9549d920b --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumOutputStream.java @@ -0,0 +1,128 @@ +/** + * 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; + private final long segmentTxId; + private final int writeTimeoutMs; + + public QuorumOutputStream(AsyncLoggerSet loggers, + long txId, int outputBufferCapacity, + int writeTimeoutMs) throws IOException { + super(); + this.buf = new EditsDoubleBuffer(outputBufferCapacity); + this.loggers = loggers; + this.segmentTxId = txId; + this.writeTimeoutMs = writeTimeoutMs; + } + + @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(boolean durable) throws IOException { + int numReadyBytes = buf.countReadyBytes(); + if (numReadyBytes > 0) { + int numReadyTxns = buf.countReadyTxns(); + long firstTxToFlush = buf.getFirstReadyTxId(); + + assert numReadyTxns > 0; + + // Copy from our double-buffer into a new byte array. This is for + // two reasons: + // 1) The IPC code has no way of specifying to send only a slice of + // a larger array. + // 2) because the calls to the underlying nodes are asynchronous, we + // need a defensive copy to avoid accidentally mutating the buffer + // before it is sent. + DataOutputBuffer bufToSend = new DataOutputBuffer(numReadyBytes); + buf.flushTo(bufToSend); + assert bufToSend.getLength() == numReadyBytes; + byte[] data = bufToSend.getData(); + assert data.length == bufToSend.getLength(); + + QuorumCall qcall = loggers.sendEdits( + segmentTxId, firstTxToFlush, + numReadyTxns, data); + loggers.waitForWriteQuorum(qcall, writeTimeoutMs, "sendEdits"); + + // Since we successfully wrote this batch, let the loggers know. Any future + // RPCs will thus let the loggers know of the most recent transaction, even + // if a logger has fallen behind. + loggers.setCommittedTxId(firstTxToFlush + numReadyTxns - 1); + } + } + + @Override + public String generateHtmlReport() { + StringBuilder sb = new StringBuilder(); + sb.append("Writing segment beginning at txid " + segmentTxId + "
\n"); + loggers.appendHtmlReport(sb); + return sb.toString(); + } + + @Override + public String toString() { + return "QuorumOutputStream starting at txid " + segmentTxId; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/SegmentRecoveryComparator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/SegmentRecoveryComparator.java new file mode 100644 index 00000000000..61b60aaad87 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/SegmentRecoveryComparator.java @@ -0,0 +1,91 @@ +/** + * 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.Comparator; +import java.util.Map.Entry; + +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ComparisonChain; +import com.google.common.primitives.Booleans; + +/** + * Compares responses to the prepareRecovery RPC. This is responsible for + * determining the correct length to recover. + */ +class SegmentRecoveryComparator + implements Comparator> { + + static final SegmentRecoveryComparator INSTANCE = new SegmentRecoveryComparator(); + + @Override + public int compare( + Entry a, + Entry b) { + + PrepareRecoveryResponseProto r1 = a.getValue(); + PrepareRecoveryResponseProto r2 = b.getValue(); + + // A response that has data for a segment is always better than one + // that doesn't. + if (r1.hasSegmentState() != r2.hasSegmentState()) { + return Booleans.compare(r1.hasSegmentState(), r2.hasSegmentState()); + } + + if (!r1.hasSegmentState()) { + // Neither has a segment, so neither can be used for recover. + // Call them equal. + return 0; + } + + // They both have a segment. + SegmentStateProto r1Seg = r1.getSegmentState(); + SegmentStateProto r2Seg = r2.getSegmentState(); + + Preconditions.checkArgument(r1Seg.getStartTxId() == r2Seg.getStartTxId(), + "Should only be called with responses for corresponding segments: " + + "%s and %s do not have the same start txid.", r1, r2); + + // If one is in-progress but the other is finalized, + // the finalized one is greater. + if (r1Seg.getIsInProgress() != r2Seg.getIsInProgress()) { + return Booleans.compare(!r1Seg.getIsInProgress(), !r2Seg.getIsInProgress()); + } + + if (!r1Seg.getIsInProgress()) { + // If both are finalized, they should match lengths + if (r1Seg.getEndTxId() != r2Seg.getEndTxId()) { + throw new AssertionError("finalized segs with different lengths: " + + r1 + ", " + r2); + } + return 0; + } + + // Both are in-progress. + long r1SeenEpoch = Math.max(r1.getAcceptedInEpoch(), r1.getLastWriterEpoch()); + long r2SeenEpoch = Math.max(r2.getAcceptedInEpoch(), r2.getLastWriterEpoch()); + + return ComparisonChain.start() + .compare(r1SeenEpoch, r2SeenEpoch) + .compare(r1.getSegmentState().getEndTxId(), r2.getSegmentState().getEndTxId()) + .result(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/JournalNotFormattedException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/JournalNotFormattedException.java new file mode 100644 index 00000000000..874d6481f8e --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/JournalNotFormattedException.java @@ -0,0 +1,35 @@ +/** + * 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; +import java.io.IOException; + +/** + * Exception indicating that a call has been made to a JournalNode + * which is not yet formatted. + */ +@InterfaceAudience.Private +public class JournalNotFormattedException extends IOException { + private static final long serialVersionUID = 1L; + + public JournalNotFormattedException(String msg) { + super(msg); + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/JournalOutOfSyncException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/JournalOutOfSyncException.java new file mode 100644 index 00000000000..15a972c85f0 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/JournalOutOfSyncException.java @@ -0,0 +1,32 @@ +/** + * 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 org.apache.hadoop.classification.InterfaceAudience; + +@InterfaceAudience.Private +public class JournalOutOfSyncException extends IOException { + private static final long serialVersionUID = 1L; + + public JournalOutOfSyncException(String msg) { + super(msg); + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java new file mode 100644 index 00000000000..769d0848bd1 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/QJournalProtocol.java @@ -0,0 +1,143 @@ +/** + * 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.namenode.JournalManager; +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; + + /** + * @return true if the given journal has been formatted and + * contains valid data. + */ + public boolean isFormatted(String journalId) throws IOException; + + /** + * Get the current state of the journal, including the most recent + * epoch number and the HTTP port. + */ + public GetJournalStateResponseProto getJournalState(String journalId) + throws IOException; + + /** + * Format the underlying storage for the given namespace. + */ + public void format(String journalId, + NamespaceInfo nsInfo) 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 segmentTxId, + long firstTxnId, + int numTxns, + byte[] records) throws IOException; + + + /** + * Heartbeat. + * This is a no-op on the server, except that it verifies that the + * caller is in fact still the active writer, and provides up-to-date + * information on the most recently committed txid. + */ + public void heartbeat(RequestInfo reqInfo) 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 the 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; + + /** + * @throws IOException + * @see JournalManager#purgeLogsOlderThan(long) + */ + public void purgeLogsOlderThan(RequestInfo requestInfo, long minTxIdToKeep) + throws IOException; + + /** + * @param jid the journal from which to enumerate edits + * @param sinceTxId the first transaction which the client cares about + * @return a list of edit log segments since the given transaction ID. + */ + public GetEditLogManifestResponseProto getEditLogManifest( + String jid, long sinceTxId) throws IOException; + + /** + * Begin the recovery process for a given segment. See the HDFS-3077 + * design document for details. + */ + public PrepareRecoveryResponseProto prepareRecovery(RequestInfo reqInfo, + long segmentTxId) throws IOException; + + /** + * Accept a proposed recovery for the given transaction ID. + */ + public void acceptRecovery(RequestInfo reqInfo, + SegmentStateProto stateToAccept, URL fromUrl) throws IOException; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/RequestInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/RequestInfo.java new file mode 100644 index 00000000000..b2167a5e41e --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocol/RequestInfo.java @@ -0,0 +1,65 @@ +/** + * 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; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; + +@InterfaceAudience.Private +public class RequestInfo { + private String jid; + private long epoch; + private long ipcSerialNumber; + private long committedTxId; + + public RequestInfo(String jid, long epoch, long ipcSerialNumber, + long committedTxId) { + this.jid = jid; + this.epoch = epoch; + this.ipcSerialNumber = ipcSerialNumber; + this.committedTxId = committedTxId; + } + + 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; + } + + public long getCommittedTxId() { + return committedTxId; + } + + public boolean hasCommittedTxId() { + return (committedTxId != HdfsConstants.INVALID_TXID); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolPB.java new file mode 100644 index 00000000000..240bbfbc88f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolPB.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.qjournal.protocolPB; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.QJournalProtocolService; +import org.apache.hadoop.ipc.ProtocolInfo; +import org.apache.hadoop.security.KerberosInfo; + +/** + * Protocol used to journal edits to a JournalNode participating + * in the quorum journal. + * Note: This extends the protocolbuffer service based interface to + * add annotations required for security. + */ +@KerberosInfo( + serverPrincipal = DFSConfigKeys.DFS_JOURNALNODE_USER_NAME_KEY, + clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY) +@ProtocolInfo(protocolName = + "org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol", + protocolVersion = 1) +@InterfaceAudience.Private +public interface QJournalProtocolPB extends + QJournalProtocolService.BlockingInterface { +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java new file mode 100644 index 00000000000..a232331b0b3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java @@ -0,0 +1,239 @@ +/** + * 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.protocol.HdfsConstants; +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.FormatRequestProto; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FormatResponseProto; +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.HeartbeatRequestProto; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.HeartbeatResponseProto; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.IsFormattedRequestProto; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.IsFormattedResponseProto; +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.PurgeLogsRequestProto; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PurgeLogsResponseProto; +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 IsFormattedResponseProto isFormatted(RpcController controller, + IsFormattedRequestProto request) throws ServiceException { + try { + boolean ret = impl.isFormatted( + convert(request.getJid())); + return IsFormattedResponseProto.newBuilder() + .setIsFormatted(ret) + .build(); + } catch (IOException ioe) { + throw new ServiceException(ioe); + } + } + + + @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); + } + } + + public FormatResponseProto format(RpcController controller, + FormatRequestProto request) throws ServiceException { + try { + impl.format(request.getJid().getIdentifier(), + PBHelper.convert(request.getNsInfo())); + return FormatResponseProto.getDefaultInstance(); + } 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.getSegmentTxnId(), req.getFirstTxnId(), + req.getNumTxns(), req.getRecords().toByteArray()); + } catch (IOException e) { + throw new ServiceException(e); + } + return JournalResponseProto.newBuilder().build(); + } + + /** @see JournalProtocol#heartbeat */ + @Override + public HeartbeatResponseProto heartbeat(RpcController controller, + HeartbeatRequestProto req) throws ServiceException { + try { + impl.heartbeat(convert(req.getReqInfo())); + } catch (IOException e) { + throw new ServiceException(e); + } + return HeartbeatResponseProto.getDefaultInstance(); + } + + /** @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 PurgeLogsResponseProto purgeLogs(RpcController controller, + PurgeLogsRequestProto req) throws ServiceException { + try { + impl.purgeLogsOlderThan(convert(req.getReqInfo()), + req.getMinTxIdToKeep()); + } catch (IOException e) { + throw new ServiceException(e); + } + return PurgeLogsResponseProto.getDefaultInstance(); + } + + @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(), + reqInfo.hasCommittedTxId() ? + reqInfo.getCommittedTxId() : HdfsConstants.INVALID_TXID); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java new file mode 100644 index 00000000000..290a62aa1d2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolTranslatorPB.java @@ -0,0 +1,278 @@ +/** + * 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.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.FormatRequestProto; +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.HeartbeatRequestProto; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.IsFormattedRequestProto; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.IsFormattedResponseProto; +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.PurgeLogsRequestProto; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.RequestInfoProto; +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.ipc.ProtobufHelper; +import org.apache.hadoop.ipc.ProtocolMetaInterface; +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.ipc.RpcClientUtil; + +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 boolean isFormatted(String journalId) throws IOException { + try { + IsFormattedRequestProto req = IsFormattedRequestProto.newBuilder() + .setJid(convertJournalId(journalId)) + .build(); + IsFormattedResponseProto resp = rpcProxy.isFormatted( + NULL_CONTROLLER, req); + return resp.getIsFormatted(); + } catch (ServiceException e) { + throw ProtobufHelper.getRemoteException(e); + } + } + + @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 void format(String jid, NamespaceInfo nsInfo) throws IOException { + try { + FormatRequestProto req = FormatRequestProto.newBuilder() + .setJid(convertJournalId(jid)) + .setNsInfo(PBHelper.convert(nsInfo)) + .build(); + rpcProxy.format(NULL_CONTROLLER, req); + } catch (ServiceException e) { + throw ProtobufHelper.getRemoteException(e); + } + } + + @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 segmentTxId, long firstTxnId, int numTxns, + byte[] records) throws IOException { + JournalRequestProto req = JournalRequestProto.newBuilder() + .setReqInfo(convert(reqInfo)) + .setSegmentTxnId(segmentTxId) + .setFirstTxnId(firstTxnId) + .setNumTxns(numTxns) + .setRecords(PBHelper.getByteString(records)) + .build(); + try { + rpcProxy.journal(NULL_CONTROLLER, req); + } catch (ServiceException e) { + throw ProtobufHelper.getRemoteException(e); + } + } + + @Override + public void heartbeat(RequestInfo reqInfo) throws IOException { + try { + rpcProxy.heartbeat(NULL_CONTROLLER, HeartbeatRequestProto.newBuilder() + .setReqInfo(convert(reqInfo)) + .build()); + } catch (ServiceException e) { + throw ProtobufHelper.getRemoteException(e); + } + } + + private QJournalProtocolProtos.RequestInfoProto convert( + RequestInfo reqInfo) { + RequestInfoProto.Builder builder = RequestInfoProto.newBuilder() + .setJournalId(convertJournalId(reqInfo.getJournalId())) + .setEpoch(reqInfo.getEpoch()) + .setIpcSerialNumber(reqInfo.getIpcSerialNumber()); + if (reqInfo.hasCommittedTxId()) { + builder.setCommittedTxId(reqInfo.getCommittedTxId()); + } + return builder.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 void purgeLogsOlderThan(RequestInfo reqInfo, long minTxIdToKeep) + throws IOException { + PurgeLogsRequestProto req = PurgeLogsRequestProto.newBuilder() + .setReqInfo(convert(reqInfo)) + .setMinTxIdToKeep(minTxIdToKeep) + .build(); + try { + rpcProxy.purgeLogs(NULL_CONTROLLER, req); + } catch (ServiceException e) { + throw ProtobufHelper.getRemoteException(e); + } + } + + @Override + public GetEditLogManifestResponseProto getEditLogManifest(String jid, + long sinceTxId) throws IOException { + try { + return rpcProxy.getEditLogManifest(NULL_CONTROLLER, + GetEditLogManifestRequestProto.newBuilder() + .setJid(convertJournalId(jid)) + .setSinceTxId(sinceTxId) + .build()); + } catch (ServiceException e) { + throw ProtobufHelper.getRemoteException(e); + } + } + + @Override + public PrepareRecoveryResponseProto prepareRecovery(RequestInfo reqInfo, + long segmentTxId) throws IOException { + try { + return rpcProxy.prepareRecovery(NULL_CONTROLLER, + PrepareRecoveryRequestProto.newBuilder() + .setReqInfo(convert(reqInfo)) + .setSegmentTxId(segmentTxId) + .build()); + } catch (ServiceException e) { + throw ProtobufHelper.getRemoteException(e); + } + } + + @Override + public void acceptRecovery(RequestInfo reqInfo, + SegmentStateProto stateToAccept, URL fromUrl) throws IOException { + try { + rpcProxy.acceptRecovery(NULL_CONTROLLER, + AcceptRecoveryRequestProto.newBuilder() + .setReqInfo(convert(reqInfo)) + .setStateToAccept(stateToAccept) + .setFromURL(fromUrl.toExternalForm()) + .build()); + } catch (ServiceException e) { + throw ProtobufHelper.getRemoteException(e); + } + } + + public boolean isMethodSupported(String methodName) throws IOException { + return RpcClientUtil.isMethodSupported(rpcProxy, + QJournalProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER, + RPC.getProtocolVersion(QJournalProtocolPB.class), methodName); + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/GetJournalEditServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/GetJournalEditServlet.java new file mode 100644 index 00000000000..6fac6a06861 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/GetJournalEditServlet.java @@ -0,0 +1,234 @@ +/** + * 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 java.util.HashSet; +import java.util.Set; + +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.DFSUtil; +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.SecondaryNameNode; +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.io.IOUtils; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.ServletUtil; +import org.apache.hadoop.util.StringUtils; + +/** + * This servlet is used in two cases: + *
    + *
  • The QuorumJournalManager, when reading edits, fetches the edit streams + * from the journal nodes.
  • + *
  • During edits synchronization, one journal node will fetch edits from + * another journal node.
  • + *
+ */ +@InterfaceAudience.Private +public class GetJournalEditServlet extends HttpServlet { + + private static final long serialVersionUID = -4635891628211723009L; + private static final Log LOG = LogFactory.getLog(GetJournalEditServlet.class); + + static final String STORAGEINFO_PARAM = "storageInfo"; + static final String JOURNAL_ID_PARAM = "jid"; + static final String SEGMENT_TXID_PARAM = "segmentTxId"; + + protected boolean isValidRequestor(HttpServletRequest request, Configuration conf) + throws IOException { + String remotePrincipal = request.getUserPrincipal().getName(); + String remoteShortName = request.getRemoteUser(); + if (remotePrincipal == null) { // This really shouldn't happen... + LOG.warn("Received null remoteUser while authorizing access to " + + "GetJournalEditServlet"); + return false; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Validating request made by " + remotePrincipal + + " / " + remoteShortName + ". This user is: " + + UserGroupInformation.getLoginUser()); + } + + Set validRequestors = new HashSet(); + validRequestors.addAll(DFSUtil.getAllNnPrincipals(conf)); + validRequestors.add( + SecurityUtil.getServerPrincipal(conf + .get(DFSConfigKeys.DFS_SECONDARY_NAMENODE_USER_NAME_KEY), + SecondaryNameNode.getHttpAddress(conf).getHostName())); + + // Check the full principal name of all the configured valid requestors. + for (String v : validRequestors) { + if (LOG.isDebugEnabled()) + LOG.debug("isValidRequestor is comparing to valid requestor: " + v); + if (v != null && v.equals(remotePrincipal)) { + if (LOG.isDebugEnabled()) + LOG.debug("isValidRequestor is allowing: " + remotePrincipal); + return true; + } + } + + // Additionally, we compare the short name of the requestor to this JN's + // username, because we want to allow requests from other JNs during + // recovery, but we can't enumerate the full list of JNs. + if (remoteShortName.equals( + UserGroupInformation.getLoginUser().getShortUserName())) { + if (LOG.isDebugEnabled()) + LOG.debug("isValidRequestor is allowing other JN principal: " + + remotePrincipal); + return true; + } + + if (LOG.isDebugEnabled()) + LOG.debug("isValidRequestor is rejecting: " + remotePrincipal); + return false; + } + + private boolean checkRequestorOrSendError(Configuration conf, + HttpServletRequest request, HttpServletResponse response) + throws IOException { + if (UserGroupInformation.isSecurityEnabled() + && !isValidRequestor(request, 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)) { + String msg = "This node has storage info '" + myStorageInfoString + + "' but the requesting node expected '" + + theirStorageInfoString + "'"; + + response.sendError(HttpServletResponse.SC_FORBIDDEN, msg); + LOG.warn("Received an invalid request file transfer request from " + + request.getRemoteAddr() + ": " + msg); + return false; + } + return true; + } + + @Override + public void doGet(final HttpServletRequest request, + final HttpServletResponse response) throws ServletException, IOException { + FileInputStream editFileIn = null; + 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; + + 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); + } finally { + IOUtils.closeStream(editFileIn); + } + } + + 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(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java new file mode 100644 index 00000000000..bb28e62ab03 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java @@ -0,0 +1,221 @@ +/** + * 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.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.hadoop.fs.FileUtil; +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; + +import com.google.common.collect.ImmutableList; + +/** + * 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 StorageState state; + + + private static final List CURRENT_DIR_PURGE_REGEXES = + ImmutableList.of( + Pattern.compile("edits_\\d+-(\\d+)"), + Pattern.compile("edits_inprogress_(\\d+)(?:\\..*)?")); + + private static final List PAXOS_DIR_PURGE_REGEXES = + ImmutableList.of(Pattern.compile("(\\d+)")); + + /** + * @param logDir the path to the directory in which data will be stored + * @param errorReporter a callback to report errors + * @throws IOException + */ + protected JNStorage(File logDir, StorageErrorReporter errorReporter) throws IOException { + super(NodeType.JOURNAL_NODE); + + sd = new StorageDirectory(logDir); + this.addStorageDir(sd); + this.fjm = new FileJournalManager(sd, errorReporter); + + analyzeStorage(); + } + + 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)); + } + + /** + * @param segmentTxId the first txid of the segment + * @param epoch the epoch number of the writer which is coordinating + * recovery + * @return the temporary path in which an edits log should be stored + * while it is being downloaded from a remote JournalNode + */ + File getSyncLogTemporaryFile(long segmentTxId, long epoch) { + String name = NNStorage.getInProgressEditsFileName(segmentTxId) + + ".epoch=" + epoch; + return new File(sd.getCurrentDir(), name); + } + + /** + * @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)); + } + + File getPaxosDir() { + return new File(sd.getCurrentDir(), "paxos"); + } + + /** + * Remove any log files and associated paxos files which are older than + * the given txid. + */ + void purgeDataOlderThan(long minTxIdToKeep) throws IOException { + purgeMatching(sd.getCurrentDir(), + CURRENT_DIR_PURGE_REGEXES, minTxIdToKeep); + purgeMatching(getPaxosDir(), PAXOS_DIR_PURGE_REGEXES, minTxIdToKeep); + } + + /** + * Purge files in the given directory which match any of the set of patterns. + * The patterns must have a single numeric capture group which determines + * the associated transaction ID of the file. Only those files for which + * the transaction ID is less than the minTxIdToKeep parameter + * are removed. + */ + private static void purgeMatching(File dir, List patterns, + long minTxIdToKeep) throws IOException { + + for (File f : FileUtil.listFiles(dir)) { + if (!f.isFile()) continue; + + for (Pattern p : patterns) { + Matcher matcher = p.matcher(f.getName()); + if (matcher.matches()) { + // This parsing will always succeed since the group(1) is + // /\d+/ in the regex itself. + long txid = Long.valueOf(matcher.group(1)); + if (txid < minTxIdToKeep) { + LOG.info("Purging no-longer needed file " + txid); + if (!f.delete()) { + LOG.warn("Unable to delete no-longer-needed data " + + f); + } + break; + } + } + } + } + } + + void format(NamespaceInfo nsInfo) throws IOException { + setStorageInfo(nsInfo); + LOG.info("Formatting journal storage directory " + + sd + " with nsid: " + getNamespaceID()); + // Unlock the directory before formatting, because we will + // re-analyze it after format(). The analyzeStorage() call + // below is reponsible for re-locking it. This is a no-op + // if the storage is not currently locked. + unlockAll(); + sd.clearDirectory(); + writeProperties(sd); + if (!getPaxosDir().mkdirs()) { + throw new IOException("Could not create paxos dir: " + getPaxosDir()); + } + analyzeStorage(); + } + + + void analyzeStorage() throws IOException { + this.state = sd.analyzeStorage(StartupOption.REGULAR, this); + if (state == StorageState.NORMAL) { + readProperties(sd); + } + } + + 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(); + } + + public boolean isFormatted() { + return state == StorageState.NORMAL; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java new file mode 100644 index 00000000000..9a15fc1b59d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java @@ -0,0 +1,953 @@ +/** + * 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.security.PrivilegedExceptionAction; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.qjournal.protocol.JournalNotFormattedException; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol; +import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException; +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.JournalManager; +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.BestEffortLongFile; +import org.apache.hadoop.hdfs.util.PersistentLongFile; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.ipc.Server; +import org.apache.hadoop.security.SecurityUtil; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Stopwatch; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Range; +import com.google.common.collect.Ranges; +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 long highestWrittenTxId = 0; + + private final String journalId; + + 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; + + /** + * Each IPC that comes from a given client contains a serial number + * which only increases from the client's perspective. Whenever + * we switch epochs, we reset this back to -1. Whenever an IPC + * comes from a client, we ensure that it is strictly higher + * than any previous IPC. This guards against any bugs in the IPC + * layer that would re-order IPCs or cause a stale retry from an old + * request to resurface and confuse things. + */ + private long currentEpochIpcSerial = -1; + + /** + * The epoch number of the last writer to actually write a transaction. + * This is used to differentiate log segments after a crash at the very + * beginning of a segment. See the the 'testNewerVersionOfSegmentWins' + * test case. + */ + private PersistentLongFile lastWriterEpoch; + + /** + * Lower-bound on the last committed transaction ID. This is not + * depended upon for correctness, but acts as a sanity check + * during the recovery procedures, and as a visibility mark + * for clients reading in-progress logs. + */ + private BestEffortLongFile committedTxnId; + + private static final String LAST_PROMISED_FILENAME = "last-promised-epoch"; + private static final String LAST_WRITER_EPOCH = "last-writer-epoch"; + private static final String COMMITTED_TXID_FILENAME = "committed-txid"; + + private final FileJournalManager fjm; + + private final JournalMetrics metrics; + + + Journal(File logDir, String journalId, + StorageErrorReporter errorReporter) throws IOException { + storage = new JNStorage(logDir, errorReporter); + this.journalId = journalId; + + refreshCachedData(); + + this.fjm = storage.getJournalManager(); + + this.metrics = JournalMetrics.create(this); + + EditLogFile latest = scanStorageForLatestEdits(); + if (latest != null) { + highestWrittenTxId = latest.getLastTxId(); + } + } + + /** + * Reload any data that may have been cached. This is necessary + * when we first load the Journal, but also after any formatting + * operation, since the cached data is no longer relevant. + */ + private synchronized void refreshCachedData() { + IOUtils.closeStream(committedTxnId); + + File currentDir = storage.getSingularStorageDir().getCurrentDir(); + this.lastPromisedEpoch = new PersistentLongFile( + new File(currentDir, LAST_PROMISED_FILENAME), 0); + this.lastWriterEpoch = new PersistentLongFile( + new File(currentDir, LAST_WRITER_EPOCH), 0); + this.committedTxnId = new BestEffortLongFile( + new File(currentDir, COMMITTED_TXID_FILENAME), + HdfsConstants.INVALID_TXID); + } + + /** + * Scan the local storage directory, and return the segment containing + * the highest transaction. + * @return the EditLogFile with the highest transactions, or null + * if no files exist. + */ + private synchronized EditLogFile scanStorageForLatestEdits() throws IOException { + if (!fjm.getStorageDirectory().getCurrentDir().exists()) { + return null; + } + + LOG.info("Scanning storage " + fjm); + List files = fjm.getLogFiles(0); + + while (!files.isEmpty()) { + EditLogFile latestLog = files.remove(files.size() - 1); + latestLog.validateLog(); + LOG.info("Latest log is " + latestLog); + if (latestLog.getLastTxId() == HdfsConstants.INVALID_TXID) { + // the log contains no transactions + LOG.warn("Latest log " + latestLog + " has no transactions. " + + "moving it aside and looking for previous log"); + latestLog.moveAsideEmptyFile(); + } else { + return latestLog; + } + } + + LOG.info("No files in " + fjm); + return null; + } + + /** + * 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); + LOG.info("Formatting " + this + " with namespace info: " + + nsInfo); + storage.format(nsInfo); + refreshCachedData(); + } + + /** + * Unlock and release resources. + */ + @Override // Closeable + public void close() throws IOException { + storage.close(); + + IOUtils.closeStream(committedTxnId); + } + + JNStorage getStorage() { + return storage; + } + + String getJournalId() { + return journalId; + } + + /** + * @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 { + checkFormatted(); + return lastPromisedEpoch.get(); + } + + synchronized public long getLastWriterEpoch() throws IOException { + checkFormatted(); + return lastWriterEpoch.get(); + } + + synchronized long getCommittedTxnIdForTests() throws IOException { + return committedTxnId.get(); + } + + synchronized long getCurrentLagTxns() throws IOException { + long committed = committedTxnId.get(); + if (committed == 0) { + return 0; + } + + return Math.max(committed - highestWrittenTxId, 0L); + } + + synchronized long getHighestWrittenTxId() { + return highestWrittenTxId; + } + + @VisibleForTesting + JournalMetrics getMetricsForTests() { + return metrics; + } + + /** + * 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 { + + checkFormatted(); + storage.checkConsistentNamespace(nsInfo); + + // Check that the new epoch being proposed is in fact newer than + // any other that we've promised. + if (epoch <= getLastPromisedEpoch()) { + throw new IOException("Proposed epoch " + epoch + " <= last promise " + + getLastPromisedEpoch()); + } + + updateLastPromisedEpoch(epoch); + abortCurSegment(); + + NewEpochResponseProto.Builder builder = + NewEpochResponseProto.newBuilder(); + + EditLogFile latestFile = scanStorageForLatestEdits(); + + if (latestFile != null) { + builder.setLastSegmentTxId(latestFile.getFirstTxId()); + } + + return builder.build(); + } + + private void updateLastPromisedEpoch(long newEpoch) throws IOException { + LOG.info("Updating lastPromisedEpoch from " + lastPromisedEpoch.get() + + " to " + newEpoch + " for client " + Server.getRemoteIp()); + lastPromisedEpoch.set(newEpoch); + + // Since we have a new writer, reset the IPC serial - it will start + // counting again from 0 for this writer. + currentEpochIpcSerial = -1; + } + + private void abortCurSegment() throws IOException { + if (curSegment == null) { + return; + } + + curSegment.abort(); + curSegment = null; + curSegmentTxId = HdfsConstants.INVALID_TXID; + } + + /** + * Write a batch of edits to the journal. + * {@see QJournalProtocol#journal(RequestInfo, long, long, int, byte[])} + */ + synchronized void journal(RequestInfo reqInfo, + long segmentTxId, long firstTxnId, + int numTxns, byte[] records) throws IOException { + checkFormatted(); + checkWriteRequest(reqInfo); + + checkSync(curSegment != null, + "Can't write, no segment open"); + + if (curSegmentTxId != segmentTxId) { + // Sanity check: it is possible that the writer will fail IPCs + // on both the finalize() and then the start() of the next segment. + // This could cause us to continue writing to an old segment + // instead of rolling to a new one, which breaks one of the + // invariants in the design. If it happens, abort the segment + // and throw an exception. + JournalOutOfSyncException e = new JournalOutOfSyncException( + "Writer out of sync: it thinks it is writing segment " + segmentTxId + + " but current segment is " + curSegmentTxId); + abortCurSegment(); + throw e; + } + + checkSync(nextTxId == firstTxnId, + "Can't write txid " + firstTxnId + " expecting nextTxId=" + nextTxId); + + long lastTxnId = firstTxnId + numTxns - 1; + if (LOG.isTraceEnabled()) { + LOG.trace("Writing txid " + firstTxnId + "-" + lastTxnId); + } + + // If the edit has already been marked as committed, we know + // it has been fsynced on a quorum of other nodes, and we are + // "catching up" with the rest. Hence we do not need to fsync. + boolean isLagging = lastTxnId <= committedTxnId.get(); + boolean shouldFsync = !isLagging; + + curSegment.writeRaw(records, 0, records.length); + curSegment.setReadyToFlush(); + Stopwatch sw = new Stopwatch(); + sw.start(); + curSegment.flush(shouldFsync); + sw.stop(); + + metrics.addSync(sw.elapsedTime(TimeUnit.MICROSECONDS)); + + if (isLagging) { + // This batch of edits has already been committed on a quorum of other + // nodes. So, we are in "catch up" mode. This gets its own metric. + metrics.batchesWrittenWhileLagging.incr(1); + } + + metrics.batchesWritten.incr(1); + metrics.bytesWritten.incr(records.length); + metrics.txnsWritten.incr(numTxns); + + highestWrittenTxId = lastTxnId; + nextTxId = lastTxnId + 1; + } + + public void heartbeat(RequestInfo reqInfo) throws IOException { + checkRequest(reqInfo); + } + + /** + * 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()); + } else if (reqInfo.getEpoch() > lastPromisedEpoch.get()) { + // A newer client has arrived. Fence any previous writers by updating + // the promise. + updateLastPromisedEpoch(reqInfo.getEpoch()); + } + + // Ensure that the IPCs are arriving in-order as expected. + checkSync(reqInfo.getIpcSerialNumber() > currentEpochIpcSerial, + "IPC serial %s from client %s was not higher than prior highest " + + "IPC serial %s", reqInfo.getIpcSerialNumber(), + Server.getRemoteIp(), + currentEpochIpcSerial); + currentEpochIpcSerial = reqInfo.getIpcSerialNumber(); + + if (reqInfo.hasCommittedTxId()) { + Preconditions.checkArgument( + reqInfo.getCommittedTxId() >= committedTxnId.get(), + "Client trying to move committed txid backward from " + + committedTxnId.get() + " to " + reqInfo.getCommittedTxId()); + + committedTxnId.set(reqInfo.getCommittedTxId()); + } + } + + private synchronized void checkWriteRequest(RequestInfo reqInfo) throws IOException { + checkRequest(reqInfo); + + if (reqInfo.getEpoch() != lastWriterEpoch.get()) { + throw new IOException("IPC's epoch " + reqInfo.getEpoch() + + " is not the current writer epoch " + + lastWriterEpoch.get()); + } + } + + public synchronized boolean isFormatted() { + return storage.isFormatted(); + } + + private void checkFormatted() throws JournalNotFormattedException { + if (!isFormatted()) { + throw new JournalNotFormattedException("Journal " + + storage.getSingularStorageDir() + " not formatted"); + } + } + + /** + * @throws JournalOutOfSyncException if the given expression is not true. + * The message of the exception is formatted using the 'msg' and + * 'formatArgs' parameters. + */ + private void checkSync(boolean expression, String msg, + Object... formatArgs) throws JournalOutOfSyncException { + if (!expression) { + throw new JournalOutOfSyncException(String.format(msg, formatArgs)); + } + } + + /** + * @throws AssertionError if the given expression is not true. + * The message of the exception is formatted using the 'msg' and + * 'formatArgs' parameters. + * + * This should be used in preference to Java's built-in assert in + * non-performance-critical paths, where a failure of this invariant + * might cause the protocol to lose data. + */ + private void alwaysAssert(boolean expression, String msg, + Object... formatArgs) { + if (!expression) { + throw new AssertionError(String.format(msg, formatArgs)); + } + } + + /** + * 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; + checkFormatted(); + checkRequest(reqInfo); + + if (curSegment != null) { + LOG.warn("Client is requesting a new log segment " + txid + + " though we are already writing " + curSegment + ". " + + "Aborting the current segment in order to begin the new one."); + // The writer may have lost a connection to us and is now + // re-connecting after the connection came back. + // We should abort our own old segment. + abortCurSegment(); + } + + // Paranoid sanity check: we should never overwrite a finalized log file. + // Additionally, if it's in-progress, it should have at most 1 transaction. + // This can happen if the writer crashes exactly at the start of a segment. + EditLogFile existing = fjm.getLogFile(txid); + if (existing != null) { + if (!existing.isInProgress()) { + throw new IllegalStateException("Already have a finalized segment " + + existing + " beginning at " + txid); + } + + // If it's in-progress, it should only contain one transaction, + // because the "startLogSegment" transaction is written alone at the + // start of each segment. + existing.validateLog(); + if (existing.getLastTxId() != existing.getFirstTxId()) { + throw new IllegalStateException("The log file " + + existing + " seems to contain valid transactions"); + } + } + + long curLastWriterEpoch = lastWriterEpoch.get(); + if (curLastWriterEpoch != reqInfo.getEpoch()) { + LOG.info("Updating lastWriterEpoch from " + curLastWriterEpoch + + " to " + reqInfo.getEpoch() + " for client " + + Server.getRemoteIp()); + lastWriterEpoch.set(reqInfo.getEpoch()); + } + + // The fact that we are starting a segment at this txid indicates + // that any previous recovery for this same segment was aborted. + // Otherwise, no writer would have started writing. So, we can + // remove the record of the older segment here. + purgePaxosDecision(txid); + + 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 { + checkFormatted(); + checkRequest(reqInfo); + + boolean needsValidation = true; + + // Finalizing the log that the writer was just writing. + if (startTxId == curSegmentTxId) { + if (curSegment != null) { + curSegment.close(); + curSegment = null; + curSegmentTxId = HdfsConstants.INVALID_TXID; + } + + checkSync(nextTxId == endTxId + 1, + "Trying to finalize in-progress log segment %s to end at " + + "txid %s but only written up to txid %s", + startTxId, endTxId, nextTxId - 1); + // No need to validate the edit log if the client is finalizing + // the log segment that it was just writing to. + needsValidation = false; + } + + FileJournalManager.EditLogFile elf = fjm.getLogFile(startTxId); + if (elf == null) { + throw new JournalOutOfSyncException("No log file to finalize at " + + "transaction ID " + startTxId); + } + + if (elf.isInProgress()) { + if (needsValidation) { + LOG.info("Validating log segment " + elf.getFile() + " about to be " + + "finalized"); + elf.validateLog(); + + checkSync(elf.getLastTxId() == endTxId, + "Trying to finalize in-progress log segment %s to end at " + + "txid %s but log %s on disk only contains up to txid %s", + startTxId, endTxId, elf.getFile(), elf.getLastTxId()); + } + fjm.finalizeLogSegment(startTxId, endTxId); + } else { + Preconditions.checkArgument(endTxId == elf.getLastTxId(), + "Trying to re-finalize already finalized log " + + elf + " with different endTxId " + endTxId); + } + + // Once logs are finalized, a different length will never be decided. + // During recovery, we treat a finalized segment the same as an accepted + // recovery. Thus, we no longer need to keep track of the previously- + // accepted decision. The existence of the finalized log segment is enough. + purgePaxosDecision(elf.getFirstTxId()); + } + + /** + * @see JournalManager#purgeLogsOlderThan(long) + */ + public synchronized void purgeLogsOlderThan(RequestInfo reqInfo, + long minTxIdToKeep) throws IOException { + checkFormatted(); + checkRequest(reqInfo); + + storage.purgeDataOlderThan(minTxIdToKeep); + } + + /** + * Remove the previously-recorded 'accepted recovery' information + * for a given log segment, once it is no longer necessary. + * @param segmentTxId the transaction ID to purge + * @throws IOException if the file could not be deleted + */ + private void purgePaxosDecision(long segmentTxId) throws IOException { + File paxosFile = storage.getPaxosFile(segmentTxId); + if (paxosFile.exists()) { + if (!paxosFile.delete()) { + throw new IOException("Unable to delete paxos file " + paxosFile); + } + } + } + + /** + * @see QJournalProtocol#getEditLogManifest(String, long) + */ + public RemoteEditLogManifest getEditLogManifest(long sinceTxId) + throws IOException { + // No need to checkRequest() here - anyone may ask for the list + // of segments. + checkFormatted(); + + 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) { + LOG.info("Edit log file " + elf + " appears to be empty. " + + "Moving it aside..."); + elf.moveAsideEmptyFile(); + return null; + } + SegmentStateProto ret = SegmentStateProto.newBuilder() + .setStartTxId(segmentTxId) + .setEndTxId(elf.getLastTxId()) + .setIsInProgress(elf.isInProgress()) + .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 { + checkFormatted(); + checkRequest(reqInfo); + + abortCurSegment(); + + PrepareRecoveryResponseProto.Builder builder = + PrepareRecoveryResponseProto.newBuilder(); + + PersistedRecoveryPaxosData previouslyAccepted = getPersistedPaxosData(segmentTxId); + completeHalfDoneAcceptRecovery(previouslyAccepted); + + SegmentStateProto segInfo = getSegmentInfo(segmentTxId); + boolean hasFinalizedSegment = segInfo != null && !segInfo.getIsInProgress(); + + if (previouslyAccepted != null && !hasFinalizedSegment) { + SegmentStateProto acceptedState = previouslyAccepted.getSegmentState(); + assert acceptedState.getEndTxId() == segInfo.getEndTxId() : + "prev accepted: " + TextFormat.shortDebugString(previouslyAccepted)+ "\n" + + "on disk: " + TextFormat.shortDebugString(segInfo); + + builder.setAcceptedInEpoch(previouslyAccepted.getAcceptedInEpoch()) + .setSegmentState(previouslyAccepted.getSegmentState()); + } else { + if (segInfo != null) { + builder.setSegmentState(segInfo); + } + } + + builder.setLastWriterEpoch(lastWriterEpoch.get()); + if (committedTxnId.get() != HdfsConstants.INVALID_TXID) { + builder.setLastCommittedTxId(committedTxnId.get()); + } + + 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 { + checkFormatted(); + checkRequest(reqInfo); + + abortCurSegment(); + + long segmentTxId = segment.getStartTxId(); + + // Basic sanity checks that the segment is well-formed and contains + // at least one transaction. + 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 we previously acted on acceptRecovery() from a higher-numbered writer, + // this call is out of sync. We should never actually trigger this, since the + // checkRequest() call above should filter non-increasing epoch numbers. + if (oldData != null) { + alwaysAssert(oldData.getAcceptedInEpoch() <= reqInfo.getEpoch(), + "Bad paxos transition, out-of-order epochs.\nOld: %s\nNew: %s\n", + oldData, newData); + } + + File syncedFile = null; + + SegmentStateProto currentSegment = getSegmentInfo(segmentTxId); + if (currentSegment == null || + currentSegment.getEndTxId() != segment.getEndTxId()) { + if (currentSegment == null) { + LOG.info("Synchronizing log " + TextFormat.shortDebugString(segment) + + ": no current segment in place"); + + // Update the highest txid for lag metrics + highestWrittenTxId = Math.max(segment.getEndTxId(), + highestWrittenTxId); + } else { + LOG.info("Synchronizing log " + TextFormat.shortDebugString(segment) + + ": old segment " + TextFormat.shortDebugString(currentSegment) + + " is not the right length"); + + // Paranoid sanity check: if the new log is shorter than the log we + // currently have, we should not end up discarding any transactions + // which are already Committed. + if (txnRange(currentSegment).contains(committedTxnId.get()) && + !txnRange(segment).contains(committedTxnId.get())) { + throw new AssertionError( + "Cannot replace segment " + + TextFormat.shortDebugString(currentSegment) + + " with new segment " + + TextFormat.shortDebugString(segment) + + ": would discard already-committed txn " + + committedTxnId.get()); + } + + // Another paranoid check: we should not be asked to synchronize a log + // on top of a finalized segment. + alwaysAssert(currentSegment.getIsInProgress(), + "Should never be asked to synchronize a different log on top of an " + + "already-finalized segment"); + + // If we're shortening the log, update our highest txid + // used for lag metrics. + if (txnRange(currentSegment).contains(highestWrittenTxId)) { + highestWrittenTxId = segment.getEndTxId(); + } + } + syncedFile = syncLog(reqInfo, segment, fromUrl); + + } else { + LOG.info("Skipping download of log " + + TextFormat.shortDebugString(segment) + + ": already have up-to-date logs"); + } + + // This is one of the few places in the protocol where we have a single + // RPC that results in two distinct actions: + // + // - 1) Downloads the new log segment data (above) + // - 2) Records the new Paxos data about the synchronized segment (below) + // + // These need to be treated as a transaction from the perspective + // of any external process. We do this by treating the persistPaxosData() + // success as the "commit" of an atomic transaction. If we fail before + // this point, the downloaded edit log will only exist at a temporary + // path, and thus not change any externally visible state. If we fail + // after this point, then any future prepareRecovery() call will see + // the Paxos data, and by calling completeHalfDoneAcceptRecovery() will + // roll forward the rename of the referenced log file. + // + // See also: HDFS-3955 + // + // The fault points here are exercised by the randomized fault injection + // test case to ensure that this atomic "transaction" operates correctly. + JournalFaultInjector.get().beforePersistPaxosData(); + persistPaxosData(segmentTxId, newData); + JournalFaultInjector.get().afterPersistPaxosData(); + + if (syncedFile != null) { + FileUtil.replaceFile(syncedFile, + storage.getInProgressEditLog(segmentTxId)); + } + + LOG.info("Accepted recovery for segment " + segmentTxId + ": " + + TextFormat.shortDebugString(newData)); + } + + private Range txnRange(SegmentStateProto seg) { + Preconditions.checkArgument(seg.hasEndTxId(), + "invalid segment: %s", seg); + return Ranges.closed(seg.getStartTxId(), seg.getEndTxId()); + } + + /** + * Synchronize a log segment from another JournalNode. The log is + * downloaded from the provided URL into a temporary location on disk, + * which is named based on the current request's epoch. + * + * @return the temporary location of the downloaded file + */ + private File syncLog(RequestInfo reqInfo, + final SegmentStateProto segment, final URL url) throws IOException { + final File tmpFile = storage.getSyncLogTemporaryFile( + segment.getStartTxId(), reqInfo.getEpoch()); + final List localPaths = ImmutableList.of(tmpFile); + + LOG.info("Synchronizing log " + + TextFormat.shortDebugString(segment) + " from " + url); + SecurityUtil.doAsLoginUser( + new PrivilegedExceptionAction() { + @Override + public Void run() throws IOException { + boolean success = false; + try { + TransferFsImage.doGetUrl(url, localPaths, storage, true); + assert tmpFile.exists(); + success = true; + } finally { + if (!success) { + if (!tmpFile.delete()) { + LOG.warn("Failed to delete temporary file " + tmpFile); + } + } + } + return null; + } + }); + return tmpFile; + } + + + /** + * In the case the node crashes in between downloading a log segment + * and persisting the associated paxos recovery data, the log segment + * will be left in its temporary location on disk. Given the paxos data, + * we can check if this was indeed the case, and "roll forward" + * the atomic operation. + * + * See the inline comments in + * {@link #acceptRecovery(RequestInfo, SegmentStateProto, URL)} for more + * details. + * + * @throws IOException if the temporary file is unable to be renamed into + * place + */ + private void completeHalfDoneAcceptRecovery( + PersistedRecoveryPaxosData paxosData) throws IOException { + if (paxosData == null) { + return; + } + + long segmentId = paxosData.getSegmentState().getStartTxId(); + long epoch = paxosData.getAcceptedInEpoch(); + + File tmp = storage.getSyncLogTemporaryFile(segmentId, epoch); + + if (tmp.exists()) { + File dst = storage.getInProgressEditLog(segmentId); + LOG.info("Rolling forward previously half-completed synchronization: " + + tmp + " -> " + dst); + FileUtil.replaceFile(tmp, dst); + } + } + + /** + * Retrieve the persisted data for recovering the given segment from disk. + */ + private PersistedRecoveryPaxosData getPersistedPaxosData(long segmentTxId) + throws IOException { + File f = storage.getPaxosFile(segmentTxId); + if (!f.exists()) { + // Default instance has no fields filled in (they're optional) + return null; + } + + InputStream in = new FileInputStream(f); + try { + PersistedRecoveryPaxosData ret = PersistedRecoveryPaxosData.parseDelimitedFrom(in); + Preconditions.checkState(ret != null && + ret.getSegmentState().getStartTxId() == segmentTxId, + "Bad persisted data for segment %s: %s", + segmentTxId, ret); + return ret; + } finally { + IOUtils.closeStream(in); + } + } + + /** + * Persist data for recovering the given segment from disk. + */ + private void persistPaxosData(long segmentTxId, + PersistedRecoveryPaxosData newData) throws IOException { + File f = storage.getPaxosFile(segmentTxId); + boolean success = false; + AtomicFileOutputStream fos = new AtomicFileOutputStream(f); + try { + newData.writeDelimitedTo(fos); + fos.write('\n'); + // Write human-readable data after the protobuf. This is only + // to assist in debugging -- it's not parsed at all. + OutputStreamWriter writer = new OutputStreamWriter(fos); + + writer.write(String.valueOf(newData)); + writer.write('\n'); + writer.flush(); + + fos.flush(); + success = true; + } finally { + if (success) { + IOUtils.closeStream(fos); + } else { + fos.abort(); + } + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalFaultInjector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalFaultInjector.java new file mode 100644 index 00000000000..cefb7b52944 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalFaultInjector.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.server; + +import java.io.IOException; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.hadoop.classification.InterfaceAudience; + +/** + * Used for injecting faults in QuorumJournalManager tests. + * Calls into this are a no-op in production code. + */ +@VisibleForTesting +@InterfaceAudience.Private +public class JournalFaultInjector { + public static JournalFaultInjector instance = new JournalFaultInjector(); + + public static JournalFaultInjector get() { + return instance; + } + + public void beforePersistPaxosData() throws IOException {} + public void afterPersistPaxosData() throws IOException {} +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalMetrics.java new file mode 100644 index 00000000000..7bbee5b5ac1 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalMetrics.java @@ -0,0 +1,118 @@ +/** + * 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 org.apache.hadoop.metrics2.annotation.Metric; +import org.apache.hadoop.metrics2.annotation.Metrics; +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.metrics2.lib.MetricsRegistry; +import org.apache.hadoop.metrics2.lib.MutableCounterLong; +import org.apache.hadoop.metrics2.lib.MutableQuantiles; + +/** + * The server-side metrics for a journal from the JournalNode's + * perspective. + */ +@Metrics(about="Journal metrics", context="dfs") +class JournalMetrics { + final MetricsRegistry registry = new MetricsRegistry("JournalNode"); + + @Metric("Number of batches written since startup") + MutableCounterLong batchesWritten; + + @Metric("Number of txns written since startup") + MutableCounterLong txnsWritten; + + @Metric("Number of bytes written since startup") + MutableCounterLong bytesWritten; + + @Metric("Number of batches written where this node was lagging") + MutableCounterLong batchesWrittenWhileLagging; + + private final int[] QUANTILE_INTERVALS = new int[] { + 1*60, // 1m + 5*60, // 5m + 60*60 // 1h + }; + + MutableQuantiles[] syncsQuantiles; + + private final Journal journal; + + JournalMetrics(Journal journal) { + this.journal = journal; + + syncsQuantiles = new MutableQuantiles[QUANTILE_INTERVALS.length]; + for (int i = 0; i < syncsQuantiles.length; i++) { + int interval = QUANTILE_INTERVALS[i]; + syncsQuantiles[i] = registry.newQuantiles( + "syncs" + interval + "s", + "Journal sync time", "ops", "latencyMicros", interval); + } + } + + public static JournalMetrics create(Journal j) { + JournalMetrics m = new JournalMetrics(j); + return DefaultMetricsSystem.instance().register( + m.getName(), null, m); + } + + String getName() { + return "Journal-" + journal.getJournalId(); + } + + @Metric("Current writer's epoch") + public long getLastWriterEpoch() { + try { + return journal.getLastWriterEpoch(); + } catch (IOException e) { + return -1L; + } + } + + @Metric("Last accepted epoch") + public long getLastPromisedEpoch() { + try { + return journal.getLastPromisedEpoch(); + } catch (IOException e) { + return -1L; + } + } + + @Metric("The highest txid stored on this JN") + public long getLastWrittenTxId() { + return journal.getHighestWrittenTxId(); + } + + @Metric("Number of transactions that this JN is lagging") + public long getCurrentLagTxns() { + try { + return journal.getCurrentLagTxns(); + } catch (IOException e) { + return -1L; + } + } + + void addSync(long us) { + for (MutableQuantiles q : syncsQuantiles) { + q.add(us); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java new file mode 100644 index 00000000000..e70f4690efc --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java @@ -0,0 +1,235 @@ +/** + * 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.security.SecurityUtil; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; + +/** + * The JournalNode is a daemon which allows namenodes using + * the QuorumJournalManager to log and retrieve edits stored + * remotely. It is a thin wrapper around a local edit log + * directory with the addition of facilities to participate + * in the quorum protocol. + */ +@InterfaceAudience.Private +public class JournalNode implements Tool, Configurable { + public static final Log LOG = LogFactory.getLog(JournalNode.class); + private Configuration conf; + private JournalNodeRpcServer rpcServer; + private JournalNodeHttpServer httpServer; + private Map journalsById = Maps.newHashMap(); + + private File localDir; + + static { + HdfsConfiguration.init(); + } + + /** + * When stopped, the daemon will exit with this code. + */ + private int resultCode = 0; + + synchronized Journal getOrCreateJournal(String jid) throws IOException { + 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, jid, new ErrorReporter()); + journalsById.put(jid, journal); + } + + return journal; + } + + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + this.localDir = new File( + conf.get(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, + DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_DEFAULT).trim()); + } + + private static void validateAndCreateJournalDir(File dir) throws IOException { + if (!dir.isAbsolute()) { + throw new IllegalArgumentException( + "Journal dir '" + dir + "' should be an absolute path"); + } + + if (!dir.exists() && !dir.mkdirs()) { + throw new IOException("Could not create journal dir '" + + dir + "'"); + } else if (!dir.isDirectory()) { + throw new IOException("Journal directory '" + dir + "' is not " + + "a directory"); + } + + if (!dir.canWrite()) { + throw new IOException("Unable to write to journal dir '" + + dir + "'"); + } + } + + + @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"); + + validateAndCreateJournalDir(localDir); + + DefaultMetricsSystem.initialize("JournalNode"); + JvmMetrics.create("JournalNode", + conf.get(DFSConfigKeys.DFS_METRICS_SESSION_ID_KEY), + DefaultMetricsSystem.instance()); + + InetSocketAddress socAddr = JournalNodeRpcServer.getAddress(conf); + SecurityUtil.login(conf, DFSConfigKeys.DFS_JOURNALNODE_KEYTAB_FILE_KEY, + DFSConfigKeys.DFS_JOURNALNODE_USER_NAME_KEY, socAddr.getHostName()); + + httpServer = new JournalNodeHttpServer(conf, this); + httpServer.start(); + + rpcServer = new JournalNodeRpcServer(conf, this); + rpcServer.start(); + } + + public boolean isStarted() { + return rpcServer != null; + } + + /** + * @return the address the IPC server is bound to + */ + public InetSocketAddress getBoundIpcAddress() { + return rpcServer.getAddress(); + } + + + public InetSocketAddress getBoundHttpAddress() { + return httpServer.getAddress(); + } + + + /** + * Stop the daemon with the given status code + * @param rc the status code with which to exit (non-zero + * should indicate an error) + */ + public void stop(int rc) { + this.resultCode = rc; + + if (rpcServer != null) { + rpcServer.stop(); + } + + if (httpServer != null) { + try { + httpServer.stop(); + } catch (IOException ioe) { + LOG.warn("Unable to stop HTTP server for " + this, ioe); + } + } + + for (Journal j : journalsById.values()) { + IOUtils.cleanup(LOG, j); + } + } + + /** + * Wait for the daemon to exit. + * @return the result code (non-zero if error) + */ + int join() throws InterruptedException { + if (rpcServer != null) { + rpcServer.join(); + } + return resultCode; + } + + public void stopAndJoin(int rc) throws InterruptedException { + stop(rc); + join(); + } + + /** + * Return the directory inside our configured storage + * dir which corresponds to a given journal. + * @param jid the journal identifier + * @return the file, which may or may not exist yet + */ + private File getLogDir(String jid) { + String dir = conf.get(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, + DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_DEFAULT); + Preconditions.checkArgument(jid != null && + !jid.isEmpty(), + "bad journal identifier: %s", jid); + return new File(new File(dir), jid); + } + + + private class ErrorReporter implements StorageErrorReporter { + @Override + public void reportErrorOnFile(File f) { + LOG.fatal("Error reported on file " + f + "... exiting", + new Exception()); + stop(1); + } + } + + public static void main(String[] args) throws Exception { + System.exit(ToolRunner.run(new JournalNode(), args)); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeHttpServer.java new file mode 100644 index 00000000000..7ad6f8d11b5 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeHttpServer.java @@ -0,0 +1,129 @@ +/** + * 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_INTERNAL_SPNEGO_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.authorize.AccessControlList; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.SecurityUtil; + +/** + * 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: "+ SecurityUtil.getServerPrincipal(conf + .get(DFS_JOURNALNODE_INTERNAL_SPNEGO_USER_NAME_KEY), + bindAddr.getHostName())); + + 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_INTERNAL_SPNEGO_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) + throws IOException { + JournalNode jn = (JournalNode)context.getAttribute(JN_ATTRIBUTE_KEY); + return jn.getOrCreateJournal(jid); + } + + public static Configuration getConfFromContext(ServletContext context) { + return (Configuration) context.getAttribute(JspHelper.CURRENT_CONF); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java new file mode 100644 index 00000000000..05a49566c02 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java @@ -0,0 +1,203 @@ +/** + * 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.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.HDFSPolicyProvider; +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; + + Configuration confCopy = new Configuration(conf); + + // Ensure that nagling doesn't kick in, which could cause latency issues. + confCopy.setBoolean( + CommonConfigurationKeysPublic.IPC_SERVER_TCPNODELAY_KEY, + true); + + InetSocketAddress addr = getAddress(confCopy); + RPC.setProtocolEngine(confCopy, QJournalProtocolPB.class, + ProtobufRpcEngine.class); + QJournalProtocolServerSideTranslatorPB translator = + new QJournalProtocolServerSideTranslatorPB(this); + BlockingService service = QJournalProtocolService + .newReflectiveBlockingService(translator); + + this.server = new RPC.Builder(confCopy) + .setProtocol(QJournalProtocolPB.class) + .setInstance(service) + .setBindAddress(addr.getHostName()) + .setPort(addr.getPort()) + .setNumHandlers(HANDLER_COUNT) + .setVerbose(false) + .build(); + + // set service-level authorization security policy + if (confCopy.getBoolean( + CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) { + server.refreshServiceAcl(confCopy, new HDFSPolicyProvider()); + } + } + + void start() { + this.server.start(); + } + + public InetSocketAddress getAddress() { + return server.getListenerAddress(); + } + + void join() throws InterruptedException { + this.server.join(); + } + + void stop() { + this.server.stop(); + } + + 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 boolean isFormatted(String journalId) throws IOException { + return jn.getOrCreateJournal(journalId).isFormatted(); + } + + @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 format(String journalId, NamespaceInfo nsInfo) + throws IOException { + jn.getOrCreateJournal(journalId).format(nsInfo); + } + + @Override + public void journal(RequestInfo reqInfo, + long segmentTxId, long firstTxnId, + int numTxns, byte[] records) throws IOException { + jn.getOrCreateJournal(reqInfo.getJournalId()) + .journal(reqInfo, segmentTxId, firstTxnId, numTxns, records); + } + + @Override + public void heartbeat(RequestInfo reqInfo) throws IOException { + jn.getOrCreateJournal(reqInfo.getJournalId()) + .heartbeat(reqInfo); + } + + @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 void purgeLogsOlderThan(RequestInfo reqInfo, long minTxIdToKeep) + throws IOException { + jn.getOrCreateJournal(reqInfo.getJournalId()) + .purgeLogsOlderThan(reqInfo, minTxIdToKeep); + } + + @Override + public GetEditLogManifestResponseProto getEditLogManifest(String jid, + long sinceTxId) throws IOException { + + RemoteEditLogManifest manifest = jn.getOrCreateJournal(jid) + .getEditLogManifest(sinceTxId); + + return GetEditLogManifestResponseProto.newBuilder() + .setManifest(PBHelper.convert(manifest)) + .setHttpPort(jn.getBoundHttpAddress().getPort()) + .build(); + } + + @Override + public PrepareRecoveryResponseProto prepareRecovery(RequestInfo reqInfo, + long segmentTxId) throws IOException { + return jn.getOrCreateJournal(reqInfo.getJournalId()) + .prepareRecovery(reqInfo, segmentTxId); + } + + @Override + public void acceptRecovery(RequestInfo reqInfo, SegmentStateProto log, + URL fromUrl) throws IOException { + jn.getOrCreateJournal(reqInfo.getJournalId()) + .acceptRecovery(reqInfo, log, fromUrl); + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java index c914035230d..50f6e730408 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java @@ -39,7 +39,8 @@ public final class HdfsServerConstants { */ static public enum NodeType { NAME_NODE, - DATA_NODE; + DATA_NODE, + JOURNAL_NODE; } /** Startup options */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java index f89fbde1219..623d5499185 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java @@ -42,6 +42,8 @@ import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.VersionInfo; +import com.google.common.base.Preconditions; + /** @@ -76,7 +78,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"; @@ -752,6 +754,15 @@ public abstract class Storage extends StorageInfo { return storageDirs.get(idx); } + /** + * @return the storage directory, with the precondition that this storage + * has exactly one storage directory + */ + public StorageDirectory getSingularStorageDir() { + Preconditions.checkState(storageDirs.size() == 1); + return storageDirs.get(0); + } + protected void addStorageDir(StorageDirectory sd) { storageDirs.add(sd); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java index 5a28f7c512d..97cfe136e40 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupOutputStream.java @@ -114,7 +114,7 @@ class EditLogBackupOutputStream extends EditLogOutputStream { } @Override // EditLogOutputStream - protected void flushAndSync() throws IOException { + protected void flushAndSync(boolean durable) throws IOException { assert out.getLength() == 0 : "Output buffer is not empty"; int numReadyTxns = doubleBuf.countReadyTxns(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java index 19604154b05..76b5ff5616a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.io.InputStream; import java.net.HttpURLConnection; import java.net.URL; +import java.security.PrivilegedExceptionAction; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -367,30 +368,36 @@ public class EditLogFileInputStream extends EditLogInputStream { @Override public InputStream getInputStream() throws IOException { - HttpURLConnection connection = (HttpURLConnection) - SecurityUtil.openSecureHttpConnection(url); - - if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) { - throw new HttpGetFailedException( - "Fetch of " + url + - " failed with status code " + connection.getResponseCode() + - "\nResponse message:\n" + connection.getResponseMessage(), - connection); - } - - String contentLength = connection.getHeaderField(CONTENT_LENGTH); - if (contentLength != null) { - advertisedSize = Long.parseLong(contentLength); - if (advertisedSize <= 0) { - throw new IOException("Invalid " + CONTENT_LENGTH + " header: " + - contentLength); - } - } else { - throw new IOException(CONTENT_LENGTH + " header is not provided " + - "by the server when trying to fetch " + url); - } - - return connection.getInputStream(); + return SecurityUtil.doAsCurrentUser( + new PrivilegedExceptionAction() { + @Override + public InputStream run() throws IOException { + HttpURLConnection connection = (HttpURLConnection) + SecurityUtil.openSecureHttpConnection(url); + + if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) { + throw new HttpGetFailedException( + "Fetch of " + url + + " failed with status code " + connection.getResponseCode() + + "\nResponse message:\n" + connection.getResponseMessage(), + connection); + } + + String contentLength = connection.getHeaderField(CONTENT_LENGTH); + if (contentLength != null) { + advertisedSize = Long.parseLong(contentLength); + if (advertisedSize <= 0) { + throw new IOException("Invalid " + CONTENT_LENGTH + " header: " + + contentLength); + } + } else { + throw new IOException(CONTENT_LENGTH + " header is not provided " + + "by the server when trying to fetch " + url); + } + + return connection.getInputStream(); + } + }); } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java index f7a8b337a6d..fb11ae0a37a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java @@ -176,7 +176,7 @@ public class EditLogFileOutputStream extends EditLogOutputStream { * accumulates new log records while readyBuffer will be flushed and synced. */ @Override - public void flushAndSync() throws IOException { + public void flushAndSync(boolean durable) throws IOException { if (fp == null) { throw new IOException("Trying to use aborted output stream"); } @@ -186,7 +186,7 @@ public class EditLogFileOutputStream extends EditLogOutputStream { } preallocate(); // preallocate file if necessay doubleBuf.flushTo(fp); - if (!shouldSkipFsyncForTests) { + if (durable && !shouldSkipFsyncForTests) { fc.force(false); // metadata updates not needed } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java index cc9b62ccafb..d5b7bffd100 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java @@ -24,6 +24,7 @@ import static org.apache.hadoop.util.Time.now; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.jasper.compiler.JspUtil; /** * A generic abstract class to support journaling of edits logs into @@ -92,18 +93,24 @@ public abstract class EditLogOutputStream implements Closeable { /** * Flush and sync all data that is ready to be flush * {@link #setReadyToFlush()} into underlying persistent store. + * @param durable if true, the edits should be made truly durable before + * returning * @throws IOException */ - abstract protected void flushAndSync() throws IOException; + abstract protected void flushAndSync(boolean durable) throws IOException; /** * Flush data to persistent store. * Collect sync metrics. */ public void flush() throws IOException { + flush(true); + } + + public void flush(boolean durable) throws IOException { numSync++; long start = now(); - flushAndSync(); + flushAndSync(durable); long end = now(); totalTimeSync += (end - start); } @@ -132,4 +139,12 @@ public abstract class EditLogOutputStream implements Closeable { protected long getNumSync() { return numSync; } + + /** + * @return a short HTML snippet suitable for describing the current + * status of the stream + */ + public String generateHtmlReport() { + return JspUtil.escapeXml(this.toString()); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java index e9d22e30bd7..d6c8a84741b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode; import java.io.IOException; import java.io.OutputStream; +import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.Writer; import org.apache.hadoop.io.DataOutputBuffer; @@ -34,7 +35,8 @@ import com.google.common.base.Preconditions; * to progress concurrently to flushes without allocating new buffers each * time. */ -class EditsDoubleBuffer { +@InterfaceAudience.Private +public class EditsDoubleBuffer { private TxnBuffer bufCurrent; // current buffer for writing private TxnBuffer bufReady; // buffer ready for flushing @@ -51,11 +53,11 @@ class EditsDoubleBuffer { bufCurrent.writeOp(op); } - void writeRaw(byte[] bytes, int offset, int length) throws IOException { + public void writeRaw(byte[] bytes, int offset, int length) throws IOException { bufCurrent.write(bytes, offset, length); } - void close() throws IOException { + public void close() throws IOException { Preconditions.checkNotNull(bufCurrent); Preconditions.checkNotNull(bufReady); @@ -69,7 +71,7 @@ class EditsDoubleBuffer { bufCurrent = bufReady = null; } - void setReadyToFlush() { + public void setReadyToFlush() { assert isFlushed() : "previous data not flushed yet"; TxnBuffer tmp = bufReady; bufReady = bufCurrent; @@ -80,12 +82,12 @@ class EditsDoubleBuffer { * Writes the content of the "ready" buffer to the given output stream, * and resets it. Does not swap any buffers. */ - void flushTo(OutputStream out) throws IOException { + public void flushTo(OutputStream out) throws IOException { bufReady.writeTo(out); // write data to file bufReady.reset(); // erase all data in the buffer } - boolean shouldForceSync() { + public boolean shouldForceSync() { return bufCurrent.size() >= initBufferSize; } @@ -120,6 +122,12 @@ class EditsDoubleBuffer { return bufReady.numTxns; } + /** + * @return the number of bytes that are ready to be flushed + */ + public int countReadyBytes() { + return bufReady.size(); + } private static class TxnBuffer extends DataOutputBuffer { long firstTxId; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java index 91ac1f6fec5..2286e2ebb53 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java @@ -1171,6 +1171,7 @@ public class FSEditLog implements LogsPurgeable { journalSet.recoverUnfinalizedSegments(); } catch (IOException ex) { // All journals have failed, it is handled in logSync. + // TODO: are we sure this is OK? } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java index d9acff9944b..08ba48dfc0e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java @@ -29,6 +29,7 @@ import java.util.Collections; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; @@ -40,6 +41,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; 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; @@ -51,7 +53,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; @@ -164,7 +167,7 @@ class FileJournalManager implements JournalManager { * @return a list of remote edit logs * @throws IOException if edit logs cannot be listed. */ - List getRemoteEditLogs(long firstTxId) throws IOException { + public List getRemoteEditLogs(long firstTxId) throws IOException { File currentDir = sd.getCurrentDir(); List allLogFiles = matchEditLogs(currentDir); List ret = Lists.newArrayListWithCapacity( @@ -182,6 +185,8 @@ class FileJournalManager implements JournalManager { } } + Collections.sort(ret); + return ret; } @@ -195,7 +200,7 @@ class FileJournalManager implements JournalManager { * @throws IOException * IOException thrown for invalid logDir */ - static List matchEditLogs(File logDir) throws IOException { + public static List matchEditLogs(File logDir) throws IOException { return matchEditLogs(FileUtil.listFiles(logDir)); } @@ -223,7 +228,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"); @@ -237,15 +242,8 @@ class FileJournalManager implements JournalManager { @Override synchronized public void selectInputStreams( Collection streams, long fromTxId, - boolean inProgressOk) { - List elfs; - try { - elfs = matchEditLogs(sd.getCurrentDir()); - } catch (IOException e) { - LOG.error("error listing files in " + this + ". " + - "Skipping all edit logs in this directory.", e); - return; - } + boolean inProgressOk) throws IOException { + List elfs = matchEditLogs(sd.getCurrentDir()); LOG.debug(this + ": selecting input streams starting at " + fromTxId + (inProgressOk ? " (inProgress ok) " : " (excluding inProgress) ") + "from among " + elfs.size() + " candidate file(s)"); @@ -321,7 +319,7 @@ class FileJournalManager implements JournalManager { } } - List getLogFiles(long fromTxId) throws IOException { + public List getLogFiles(long fromTxId) throws IOException { File currentDir = sd.getCurrentDir(); List allLogFiles = matchEditLogs(currentDir); List logFiles = Lists.newArrayList(); @@ -337,6 +335,32 @@ class FileJournalManager implements JournalManager { return logFiles; } + + public EditLogFile getLogFile(long startTxId) throws IOException { + return getLogFile(sd.getCurrentDir(), startTxId); + } + + public static EditLogFile getLogFile(File dir, long startTxId) + throws IOException { + List files = matchEditLogs(dir); + List ret = Lists.newLinkedList(); + for (EditLogFile elf : files) { + if (elf.getFirstTxId() == startTxId) { + ret.add(elf); + } + } + + if (ret.isEmpty()) { + // no matches + return null; + } else if (ret.size() == 1) { + return ret.get(0); + } else { + throw new IllegalStateException("More than one log segment in " + + dir + " starting at txid " + startTxId + ": " + + Joiner.on(", ").join(ret)); + } + } @Override public String toString() { @@ -346,7 +370,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; @@ -379,17 +404,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; } @@ -402,17 +430,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; } @@ -425,7 +453,7 @@ class FileJournalManager implements JournalManager { renameSelf(".corrupt"); } - void moveAsideEmptyFile() throws IOException { + public void moveAsideEmptyFile() throws IOException { assert lastTxId == HdfsConstants.INVALID_TXID; renameSelf(".empty"); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java index cda1152aef9..872f7023d28 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java @@ -65,9 +65,11 @@ public interface JournalManager extends Closeable, FormatConfirmable, * @param inProgressOk whether or not in-progress streams should be returned * * @return a list of streams + * @throws IOException if the underlying storage has an error or is otherwise + * inaccessible */ void selectInputStreams(Collection streams, - long fromTxnId, boolean inProgressOk); + long fromTxnId, boolean inProgressOk) throws IOException; /** * Set the amount of memory that this stream should use to buffer edits diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java index db64e63874c..8ed073d10d4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java @@ -26,6 +26,7 @@ import java.util.LinkedList; import java.util.List; import java.util.PriorityQueue; import java.util.SortedSet; +import java.util.concurrent.CopyOnWriteArrayList; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -147,7 +148,7 @@ public class JournalSet implements JournalManager { return journal; } - private boolean isDisabled() { + boolean isDisabled() { return disabled; } @@ -165,8 +166,12 @@ public class JournalSet implements JournalManager { return required; } } - - private List journals = Lists.newArrayList(); + + // COW implementation is necessary since some users (eg the web ui) call + // getAllJournalStreams() and then iterate. Since this is rarely + // mutated, there is no performance concern. + private List journals = + new CopyOnWriteArrayList(); final int minimumRedundantJournals; JournalSet(int minimumRedundantResources) { @@ -242,8 +247,20 @@ public class JournalSet implements JournalManager { LOG.info("Skipping jas " + jas + " since it's disabled"); continue; } - jas.getManager().selectInputStreams(allStreams, fromTxId, inProgressOk); + try { + jas.getManager().selectInputStreams(allStreams, fromTxId, inProgressOk); + } catch (IOException ioe) { + LOG.warn("Unable to determine input streams from " + jas.getManager() + + ". Skipping.", ioe); + } } + chainAndMakeRedundantStreams(streams, allStreams, fromTxId, inProgressOk); + } + + public static void chainAndMakeRedundantStreams( + Collection outStreams, + PriorityQueue allStreams, + long fromTxId, boolean inProgressOk) { // We want to group together all the streams that start on the same start // transaction ID. To do this, we maintain an accumulator (acc) of all // the streams we've seen at a given start transaction ID. When we see a @@ -261,7 +278,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()) { @@ -272,7 +289,7 @@ public class JournalSet implements JournalManager { } } if (!acc.isEmpty()) { - streams.add(new RedundantEditLogInputStream(acc, fromTxId)); + outStreams.add(new RedundantEditLogInputStream(acc, fromTxId)); acc.clear(); } } @@ -454,12 +471,12 @@ public class JournalSet implements JournalManager { } @Override - protected void flushAndSync() throws IOException { + protected void flushAndSync(final boolean durable) throws IOException { mapJournalsAndReportErrors(new JournalClosure() { @Override public void apply(JournalAndStream jas) throws IOException { if (jas.isActive()) { - jas.getCurrentStream().flushAndSync(); + jas.getCurrentStream().flushAndSync(durable); } } }, "flushAndSync"); @@ -512,7 +529,6 @@ public class JournalSet implements JournalManager { } } - @VisibleForTesting List getAllJournalStreams() { return journals; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java index 616e8438868..fb5c88d8bbc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java @@ -722,6 +722,12 @@ public class NameNode { String namenodeId = HAUtil.getNameNodeId(conf, nsId); initializeGenericKeys(conf, nsId, namenodeId); checkAllowFormat(conf); + + if (UserGroupInformation.isSecurityEnabled()) { + InetSocketAddress socAddr = getAddress(conf); + SecurityUtil.login(conf, DFS_NAMENODE_KEYTAB_FILE_KEY, + DFS_NAMENODE_USER_NAME_KEY, socAddr.getHostName()); + } Collection nameDirsToFormat = FSNamesystem.getNamespaceDirs(conf); List sharedDirs = FSNamesystem.getSharedEditsDirs(conf); @@ -763,13 +769,13 @@ public class NameNode { } @VisibleForTesting - public static boolean initializeSharedEdits(Configuration conf) { + public static boolean initializeSharedEdits(Configuration conf) throws IOException { return initializeSharedEdits(conf, true); } @VisibleForTesting public static boolean initializeSharedEdits(Configuration conf, - boolean force) { + boolean force) throws IOException { return initializeSharedEdits(conf, force, false); } @@ -783,7 +789,7 @@ public class NameNode { * @return true if the command aborts, false otherwise */ private static boolean initializeSharedEdits(Configuration conf, - boolean force, boolean interactive) { + boolean force, boolean interactive) throws IOException { String nsId = DFSUtil.getNamenodeNameServiceId(conf); String namenodeId = HAUtil.getNameNodeId(conf, nsId); initializeGenericKeys(conf, nsId, namenodeId); @@ -794,6 +800,12 @@ public class NameNode { return false; } + if (UserGroupInformation.isSecurityEnabled()) { + InetSocketAddress socAddr = getAddress(conf); + SecurityUtil.login(conf, DFS_NAMENODE_KEYTAB_FILE_KEY, + DFS_NAMENODE_USER_NAME_KEY, socAddr.getHostName()); + } + NNStorage existingStorage = null; try { Configuration confWithoutShared = new Configuration(conf); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourcePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourcePolicy.java index 3896165ff39..03f226c23b7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourcePolicy.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourcePolicy.java @@ -41,6 +41,14 @@ final class NameNodeResourcePolicy { static boolean areResourcesAvailable( Collection resources, int minimumRedundantResources) { + + // TODO: workaround: + // - during startup, if there are no edits dirs on disk, then there is + // a call to areResourcesAvailable() with no dirs at all, which was + // previously causing the NN to enter safemode + if (resources.isEmpty()) { + return true; + } int requiredResourceCount = 0; int redundantResourceCount = 0; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java index 1ffb343fd73..07f3f5ff771 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; import org.apache.hadoop.hdfs.server.common.JspHelper; import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; +import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.apache.hadoop.http.HttpConfig; import org.apache.hadoop.io.Text; @@ -60,6 +61,8 @@ import org.apache.hadoop.util.Time; import org.apache.hadoop.util.VersionInfo; import org.znerd.xmlenc.XMLOutputter; +import com.google.common.base.Preconditions; + class NamenodeJspHelper { static String getSafeModeText(FSNamesystem fsn) { if (!fsn.isInSafeMode()) @@ -212,6 +215,52 @@ class NamenodeJspHelper { out.print("\n"); } + + /** + * Generate an HTML report containing the current status of the HDFS + * journals. + */ + void generateJournalReport(JspWriter out, NameNode nn, + HttpServletRequest request) throws IOException { + FSEditLog log = nn.getFSImage().getEditLog(); + Preconditions.checkArgument(log != null, "no edit log set in %s", nn); + + out.println("

" + nn.getRole() + " Journal Status:

"); + + out.println("Current transaction ID: " + + nn.getFSImage().getLastAppliedOrWrittenTxId() + "
"); + + + boolean openForWrite = log.isOpenForWrite(); + + out.println("
"); + out.println("\n" + + ""); + for (JournalAndStream jas : log.getJournals()) { + out.print(""); + out.print(""); + } + + out.println("
Journal ManagerState
" + jas.getManager()); + if (jas.isRequired()) { + out.print(" [required]"); + } + out.print(""); + + if (jas.isDisabled()) { + out.print("Failed"); + } else if (openForWrite) { + EditLogOutputStream elos = jas.getCurrentStream(); + if (elos != null) { + out.println(elos.generateHtmlReport()); + } else { + out.println("not currently writing"); + } + } else { + out.println("open for read"); + } + out.println("
"); + } void generateHealthReport(JspWriter out, NameNode nn, HttpServletRequest request) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java index a403706bbfe..8e0739afb1e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.namenode.ha; import java.io.IOException; import java.net.InetSocketAddress; import java.security.PrivilegedAction; +import java.security.PrivilegedExceptionAction; import java.util.Collection; import org.apache.commons.logging.Log; @@ -172,11 +173,20 @@ public class EditLogTailer { Preconditions.checkState(tailerThread == null || !tailerThread.isAlive(), "Tailer thread should not be running once failover starts"); - try { - doTailEdits(); - } catch (InterruptedException e) { - throw new IOException(e); - } + // Important to do tailing as the login user, in case the shared + // edits storage is implemented by a JournalManager that depends + // on security credentials to access the logs (eg QuorumJournalManager). + SecurityUtil.doAsLoginUser(new PrivilegedExceptionAction() { + @Override + public Void run() throws Exception { + try { + doTailEdits(); + } catch (InterruptedException e) { + throw new IOException(e); + } + return null; + } + }); } @VisibleForTesting diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java index b6f58877f2f..68fad564a80 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLog.java @@ -17,18 +17,15 @@ */ package org.apache.hadoop.hdfs.server.protocol; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.io.Writable; import com.google.common.base.Function; import com.google.common.collect.ComparisonChain; -public class RemoteEditLog implements Writable, Comparable { +public class RemoteEditLog implements Comparable { private long startTxId = HdfsConstants.INVALID_TXID; private long endTxId = HdfsConstants.INVALID_TXID; + private boolean isInProgress = false; public RemoteEditLog() { } @@ -36,6 +33,13 @@ public class RemoteEditLog implements Writable, Comparable { public RemoteEditLog(long startTxId, long endTxId) { this.startTxId = startTxId; this.endTxId = endTxId; + this.isInProgress = (endTxId == HdfsConstants.INVALID_TXID); + } + + public RemoteEditLog(long startTxId, long endTxId, boolean inProgress) { + this.startTxId = startTxId; + this.endTxId = endTxId; + this.isInProgress = inProgress; } public long getStartTxId() { @@ -45,22 +49,18 @@ public class RemoteEditLog implements Writable, Comparable { public long getEndTxId() { return endTxId; } - + + public boolean isInProgress() { + return isInProgress; + } + @Override public String toString() { - return "[" + startTxId + "," + endTxId + "]"; - } - - @Override - public void write(DataOutput out) throws IOException { - out.writeLong(startTxId); - out.writeLong(endTxId); - } - - @Override - public void readFields(DataInput in) throws IOException { - startTxId = in.readLong(); - endTxId = in.readLong(); + if (!isInProgress) { + return "[" + startTxId + "," + endTxId + "]"; + } else { + return "[" + startTxId + "-? (in-progress)]"; + } } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLogManifest.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLogManifest.java index 60e41a615e7..0b3a0318f88 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLogManifest.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RemoteEditLogManifest.java @@ -40,8 +40,8 @@ public class RemoteEditLogManifest { /** - * Check that the logs are contiguous and non-overlapping - * sequences of transactions, in sorted order + * Check that the logs are non-overlapping sequences of transactions, + * in sorted order. They do not need to be contiguous. * @throws IllegalStateException if incorrect */ private void checkState() { @@ -50,8 +50,10 @@ public class RemoteEditLogManifest { RemoteEditLog prev = null; for (RemoteEditLog log : logs) { if (prev != null) { - if (log.getStartTxId() != prev.getEndTxId() + 1) { - throw new IllegalStateException("Invalid log manifest:" + this); + if (log.getStartTxId() <= prev.getEndTxId()) { + throw new IllegalStateException( + "Invalid log manifest (log " + log + " overlaps " + prev + ")\n" + + this); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/BinaryEditsVisitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/BinaryEditsVisitor.java index 1911ef71d2c..8c4fcbbbe3a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/BinaryEditsVisitor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/BinaryEditsVisitor.java @@ -56,7 +56,7 @@ public class BinaryEditsVisitor implements OfflineEditsVisitor { @Override public void close(Throwable error) throws IOException { elfos.setReadyToFlush(); - elfos.flushAndSync(); + elfos.flushAndSync(true); elfos.close(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/BestEffortLongFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/BestEffortLongFile.java new file mode 100644 index 00000000000..292402b245f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/BestEffortLongFile.java @@ -0,0 +1,117 @@ +/** + * 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.util; + +import java.io.Closeable; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.io.IOUtils; + +import com.google.common.io.Files; +import com.google.common.primitives.Longs; + +/** + * Class that represents a file on disk which stores a single long + * value, but does not make any effort to make it truly durable. This is in + * contrast to {@link PersistentLongFile} which fsync()s the value on every + * change. + * + * This should be used for values which are updated frequently (such that + * performance is important) and not required to be up-to-date for correctness. + * + * This class also differs in that it stores the value as binary data instead + * of a textual string. + */ +@InterfaceAudience.Private +public class BestEffortLongFile implements Closeable { + + private final File file; + private final long defaultVal; + + private long value; + + private FileChannel ch = null; + + private ByteBuffer buf = ByteBuffer.allocate(Long.SIZE/8); + + public BestEffortLongFile(File file, long defaultVal) { + this.file = file; + this.defaultVal = defaultVal; + } + + public long get() throws IOException { + lazyOpen(); + return value; + } + + public void set(long newVal) throws IOException { + lazyOpen(); + buf.clear(); + buf.putLong(newVal); + buf.flip(); + IOUtils.writeFully(ch, buf, 0); + value = newVal; + } + + private void lazyOpen() throws IOException { + if (ch != null) { + return; + } + + // Load current value. + byte[] data = null; + try { + data = Files.toByteArray(file); + } catch (FileNotFoundException fnfe) { + // Expected - this will use default value. + } + + if (data != null && data.length != 0) { + if (data.length != Longs.BYTES) { + throw new IOException("File " + file + " had invalid length: " + + data.length); + } + value = Longs.fromByteArray(data); + } else { + value = defaultVal; + } + + // Now open file for future writes. + RandomAccessFile raf = new RandomAccessFile(file, "rw"); + try { + ch = raf.getChannel(); + } finally { + if (ch == null) { + IOUtils.closeStream(raf); + } + } + } + + @Override + public void close() throws IOException { + if (ch != null) { + ch.close(); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/PersistentLongFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/PersistentLongFile.java index 5e776226fa6..292d0dfe63e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/PersistentLongFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/PersistentLongFile.java @@ -57,7 +57,9 @@ public class PersistentLongFile { } public void set(long newVal) throws IOException { - writeFile(file, newVal); + if (value != newVal || !loaded) { + writeFile(file, newVal); + } value = newVal; loaded = true; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto new file mode 100644 index 00000000000..d188f2beca3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto @@ -0,0 +1,252 @@ +/** + * 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; + + // Whenever a writer makes a request, it informs + // the node of the latest committed txid. This may + // be higher than the transaction data included in the + // request itself, eg in the case that the node has + // fallen behind. + optional uint64 committedTxId = 4; +} + +message SegmentStateProto { + required uint64 startTxId = 1; + required uint64 endTxId = 2; + required bool isInProgress = 3; +} + +/** + * 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; + required uint64 segmentTxnId = 5; +} + +message JournalResponseProto { +} + +/** + * heartbeat() + */ + +message HeartbeatRequestProto { + required RequestInfoProto reqInfo = 1; +} + +message HeartbeatResponseProto { // void response +} + +/** + * 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 { +} + +/** + * purgeLogs() + */ +message PurgeLogsRequestProto { + required RequestInfoProto reqInfo = 1; + required uint64 minTxIdToKeep = 2; +} + +message PurgeLogsResponseProto { +} + +/** + * isFormatted() + */ +message IsFormattedRequestProto { + required JournalIdProto jid = 1; +} + +message IsFormattedResponseProto { + required bool isFormatted = 1; +} + +/** + * getJournalState() + */ +message GetJournalStateRequestProto { + required JournalIdProto jid = 1; +} + +message GetJournalStateResponseProto { + required uint64 lastPromisedEpoch = 1; + required uint32 httpPort = 2; +} + +/** + * format() + */ +message FormatRequestProto { + required JournalIdProto jid = 1; + required NamespaceInfoProto nsInfo = 2; +} + +message FormatResponseProto { +} + +/** + * 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; + required uint64 lastWriterEpoch = 3; + + // The highest committed txid that this logger has ever seen. + // This may be higher than the data it actually has, in the case + // that it was lagging before the old writer crashed. + optional uint64 lastCommittedTxId = 4; +} + +/** + * 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 isFormatted(IsFormattedRequestProto) returns (IsFormattedResponseProto); + + rpc getJournalState(GetJournalStateRequestProto) returns (GetJournalStateResponseProto); + + rpc newEpoch(NewEpochRequestProto) returns (NewEpochResponseProto); + + rpc format(FormatRequestProto) returns (FormatResponseProto); + + rpc journal(JournalRequestProto) returns (JournalResponseProto); + + rpc heartbeat(HeartbeatRequestProto) returns (HeartbeatResponseProto); + + rpc startLogSegment(StartLogSegmentRequestProto) + returns (StartLogSegmentResponseProto); + + rpc finalizeLogSegment(FinalizeLogSegmentRequestProto) + returns (FinalizeLogSegmentResponseProto); + + rpc purgeLogs(PurgeLogsRequestProto) + returns (PurgeLogsResponseProto); + + rpc getEditLogManifest(GetEditLogManifestRequestProto) + returns (GetEditLogManifestResponseProto); + + rpc prepareRecovery(PrepareRecoveryRequestProto) + returns (PrepareRecoveryResponseProto); + + rpc acceptRecovery(AcceptRecoveryRequestProto) + returns (AcceptRecoveryResponseProto); +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto index 70a04752ffc..924fc019243 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto @@ -302,6 +302,7 @@ message BlocksWithLocationsProto { message RemoteEditLogProto { required uint64 startTxId = 1; // Starting available edit log transaction required uint64 endTxId = 2; // Ending available edit log transaction + optional bool isInProgress = 3 [default = false]; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index ae9f5008131..153e21ba15f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -259,6 +259,11 @@ + + dfs.namenode.edits.journal-plugin.qjournal + org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager + + dfs.permissions.enabled true @@ -1118,4 +1123,21 @@ + + dfs.journalnode.rpc-address + 0.0.0.0:8485 + + The JournalNode RPC server address and port. + + + + + dfs.journalnode.http-address + 0.0.0.0:8480 + + The address and port the JournalNode web UI listens on. + If the port is 0 then the server will start on a free port. + + + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp index 3affe6b7a42..a89e0367d5c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.jsp @@ -60,8 +60,10 @@ <%= NamenodeJspHelper.getCorruptFilesWarning(fsn)%> <% healthjsp.generateHealthReport(out, nn, request); %> -
+<% healthjsp.generateJournalReport(out, nn, request); %> +
<% healthjsp.generateConfReport(out, nn, request); %> +
<% out.println(ServletUtil.htmlFooter()); %> diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/index.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/index.html new file mode 100644 index 00000000000..bc9ea422aad --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/index.html @@ -0,0 +1,29 @@ + + + +Hadoop Administration + + +

Hadoop Administration

+ + + + + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/journalstatus.jsp b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/journalstatus.jsp new file mode 100644 index 00000000000..2c6e5a35213 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/journal/journalstatus.jsp @@ -0,0 +1,42 @@ +<% +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +%> +<%@ page + contentType="text/html; charset=UTF-8" + import="org.apache.hadoop.hdfs.server.common.JspHelper" + import="org.apache.hadoop.util.ServletUtil" +%> +<%! + //for java.io.Serializable + private static final long serialVersionUID = 1L; +%> + + + + +Hadoop JournalNode + + +

JournalNode

+<%= JspHelper.getVersionTable() %> +
+ +
+Logs +<%= ServletUtil.htmlFooter() %> diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/proto-journal-web.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/proto-journal-web.xml new file mode 100644 index 00000000000..b27f82b4a46 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/proto-journal-web.xml @@ -0,0 +1,17 @@ + + + +@journal.servlet.definitions@ + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java index 6f862a32981..3e1451ce651 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java @@ -85,6 +85,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.VersionInfo; +import com.google.common.base.Charsets; import com.google.common.base.Joiner; /** Utilities for HDFS tests */ @@ -607,12 +608,21 @@ public class DFSTestUtil { IOUtils.copyBytes(is, os, s.length(), true); } - // Returns url content as string. + /** + * @return url content as string (UTF-8 encoding assumed) + */ public static String urlGet(URL url) throws IOException { + return new String(urlGetBytes(url), Charsets.UTF_8); + } + + /** + * @return URL contents as a byte array + */ + public static byte[] urlGetBytes(URL url) throws IOException { URLConnection conn = url.openConnection(); ByteArrayOutputStream out = new ByteArrayOutputStream(); IOUtils.copyBytes(conn.getInputStream(), out, 4096, true); - return out.toString(); + return out.toByteArray(); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java new file mode 100644 index 00000000000..840f4939aeb --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java @@ -0,0 +1,194 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.qjournal; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.qjournal.server.JournalNode; + +import com.google.common.base.Joiner; +import com.google.common.collect.Lists; + +public class MiniJournalCluster { + public static class Builder { + private String baseDir; + private int numJournalNodes = 3; + private boolean format = true; + private Configuration conf; + + public Builder(Configuration conf) { + this.conf = conf; + } + + public Builder baseDir(String d) { + this.baseDir = d; + return this; + } + + public Builder numJournalNodes(int n) { + this.numJournalNodes = n; + return this; + } + + public Builder format(boolean f) { + this.format = f; + return this; + } + + public MiniJournalCluster build() throws IOException { + return new MiniJournalCluster(this); + } + } + + private static final Log LOG = LogFactory.getLog(MiniJournalCluster.class); + private File baseDir; + private JournalNode nodes[]; + private InetSocketAddress ipcAddrs[]; + private InetSocketAddress httpAddrs[]; + + private MiniJournalCluster(Builder b) throws IOException { + LOG.info("Starting MiniJournalCluster with " + + b.numJournalNodes + " journal nodes"); + + if (b.baseDir != null) { + this.baseDir = new File(b.baseDir); + } else { + this.baseDir = new File(MiniDFSCluster.getBaseDirectory()); + } + + nodes = new JournalNode[b.numJournalNodes]; + ipcAddrs = new InetSocketAddress[b.numJournalNodes]; + httpAddrs = new InetSocketAddress[b.numJournalNodes]; + for (int i = 0; i < b.numJournalNodes; i++) { + if (b.format) { + File dir = getStorageDir(i); + LOG.debug("Fully deleting JN directory " + dir); + FileUtil.fullyDelete(dir); + } + nodes[i] = new JournalNode(); + nodes[i].setConf(createConfForNode(b, i)); + nodes[i].start(); + + ipcAddrs[i] = nodes[i].getBoundIpcAddress(); + httpAddrs[i] = nodes[i].getBoundHttpAddress(); + } + } + + /** + * Set up the given Configuration object to point to the set of JournalNodes + * in this cluster. + */ + public URI getQuorumJournalURI(String jid) { + List addrs = Lists.newArrayList(); + for (InetSocketAddress addr : ipcAddrs) { + addrs.add("127.0.0.1:" + addr.getPort()); + } + String addrsVal = Joiner.on(";").join(addrs); + LOG.debug("Setting logger addresses to: " + addrsVal); + try { + return new URI("qjournal://" + addrsVal + "/" + jid); + } catch (URISyntaxException e) { + throw new AssertionError(e); + } + } + + /** + * Start the JournalNodes in the cluster. + */ + public void start() throws IOException { + for (JournalNode jn : nodes) { + jn.start(); + } + } + + /** + * Shutdown all of the JournalNodes in the cluster. + * @throws IOException if one or more nodes failed to stop + */ + public void shutdown() throws IOException { + boolean failed = false; + for (JournalNode jn : nodes) { + try { + jn.stopAndJoin(0); + } catch (Exception e) { + failed = true; + LOG.warn("Unable to stop journal node " + jn, e); + } + } + if (failed) { + throw new IOException("Unable to shut down. Check log for details"); + } + } + + private Configuration createConfForNode(Builder b, int idx) { + Configuration conf = new Configuration(b.conf); + File logDir = getStorageDir(idx); + conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, logDir.toString()); + conf.set(DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_KEY, "0.0.0.0:0"); + conf.set(DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_KEY, "0.0.0.0:0"); + return conf; + } + + public File getStorageDir(int idx) { + return new File(baseDir, "journalnode-" + idx).getAbsoluteFile(); + } + + public File getCurrentDir(int idx, String jid) { + return new File(new File(getStorageDir(idx), jid), "current"); + } + + public JournalNode getJournalNode(int i) { + return nodes[i]; + } + + public void restartJournalNode(int i) throws InterruptedException, IOException { + Configuration conf = new Configuration(nodes[i].getConf()); + if (nodes[i].isStarted()) { + nodes[i].stopAndJoin(0); + } + + conf.set(DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_KEY, "127.0.0.1:" + + ipcAddrs[i].getPort()); + conf.set(DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_KEY, "127.0.0.1:" + + httpAddrs[i].getPort()); + + JournalNode jn = new JournalNode(); + jn.setConf(conf); + jn.start(); + } + + public int getQuorumSize() { + return nodes.length / 2 + 1; + } + + public int getNumNodes() { + return nodes.length; + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/QJMTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/QJMTestUtil.java new file mode 100644 index 00000000000..f06f62eba9b --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/QJMTestUtil.java @@ -0,0 +1,161 @@ +/** + * 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.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager; +import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream; +import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream; +import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp; +import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes; +import org.apache.hadoop.hdfs.server.namenode.NNStorage; +import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; +import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; +import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.io.IOUtils; + +import com.google.common.collect.Lists; + +public abstract class QJMTestUtil { + public static final NamespaceInfo FAKE_NSINFO = new NamespaceInfo( + 12345, "mycluster", "my-bp", 0L); + public static final String JID = "test-journal"; + + 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()); + } + + public static EditLogOutputStream writeSegment(MiniJournalCluster cluster, + QuorumJournalManager qjm, long 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); + return null; + } else { + return stm; + } + } + + public static void writeOp(EditLogOutputStream stm, long txid) throws IOException { + FSEditLogOp op = NameNodeAdapter.createMkdirOp("tx " + txid); + op.setTransactionId(txid); + stm.write(op); + } + + public static void writeTxns(EditLogOutputStream stm, long startTxId, int numTxns) + throws IOException { + for (long txid = startTxId; txid < startTxId + numTxns; txid++) { + writeOp(stm, txid); + } + stm.setReadyToFlush(); + stm.flush(); + } + + /** + * Verify that the given list of streams contains exactly the range of + * transactions specified, inclusive. + */ + public static void verifyEdits(List streams, + int firstTxnId, int lastTxnId) throws IOException { + + Iterator iter = streams.iterator(); + assertTrue(iter.hasNext()); + EditLogInputStream stream = iter.next(); + + for (int expected = firstTxnId; + expected <= lastTxnId; + expected++) { + + FSEditLogOp op = stream.readOp(); + while (op == null) { + assertTrue("Expected to find txid " + expected + ", " + + "but no more streams available to read from", + iter.hasNext()); + stream = iter.next(); + op = stream.readOp(); + } + + assertEquals(FSEditLogOpCodes.OP_MKDIR, op.opCode); + assertEquals(expected, op.getTransactionId()); + } + + assertNull(stream.readOp()); + assertFalse("Expected no more txns after " + lastTxnId + + " but more streams are available", iter.hasNext()); + } + + + public static 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()); + } + + public static long recoverAndReturnLastTxn(QuorumJournalManager qjm) + throws IOException { + qjm.recoverUnfinalizedSegments(); + long lastRecoveredTxn = 0; + + List streams = Lists.newArrayList(); + try { + qjm.selectInputStreams(streams, 0, false); + + for (EditLogInputStream elis : streams) { + assertTrue(elis.getFirstTxId() > lastRecoveredTxn); + lastRecoveredTxn = elis.getLastTxId(); + } + } finally { + IOUtils.cleanup(null, streams.toArray(new Closeable[0])); + } + return lastRecoveredTxn; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestMiniJournalCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestMiniJournalCluster.java new file mode 100644 index 00000000000..fbb51e12dc8 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestMiniJournalCluster.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; + +import static org.junit.Assert.*; + +import java.io.File; +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( + new File(MiniDFSCluster.getBaseDirectory() + "journalnode-0") + .getAbsolutePath(), + dir); + } finally { + c.shutdown(); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java new file mode 100644 index 00000000000..a5463d03b39 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java @@ -0,0 +1,242 @@ +/** + * 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.File; +import java.io.IOException; +import java.net.URL; +import java.util.regex.Pattern; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +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.DFSTestUtil; +import org.apache.hadoop.hdfs.HdfsConfiguration; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.server.namenode.NameNode; +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 -- both on-disk + // and QJM. + MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf) + .numDataNodes(0) + .manageNameDfsDirs(false) + .build(); + cluster.shutdown(); + + // Reformat just the on-disk portion + Configuration onDiskOnly = new Configuration(conf); + onDiskOnly.unset(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY); + NameNode.format(onDiskOnly); + + // Start the NN - should fail because the JNs are still formatted + // with the old namespace ID. + try { + cluster = new MiniDFSCluster.Builder(conf) + .numDataNodes(0) + .manageNameDfsDirs(false) + .format(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); + } + } + + @Test + public void testWebPageHasQjmInfo() 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()); + // Speed up the test + conf.setInt( + CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 1); + + MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf) + .numDataNodes(0) + .manageNameDfsDirs(false) + .build(); + try { + URL url = new URL("http://localhost:" + + NameNode.getHttpAddress(cluster.getConfiguration(0)).getPort() + + "/dfshealth.jsp"); + + cluster.getFileSystem().mkdirs(TEST_PATH); + + String contents = DFSTestUtil.urlGet(url); + assertTrue(contents.contains("QJM to [")); + assertTrue(contents.contains("Written txid 2")); + + // Stop one JN, do another txn, and make sure it shows as behind + // stuck behind the others. + mjc.getJournalNode(0).stopAndJoin(0); + + cluster.getFileSystem().delete(TEST_PATH, true); + + contents = DFSTestUtil.urlGet(url); + System.out.println(contents); + assertTrue(Pattern.compile("1 txns/\\d+ms behind").matcher(contents) + .find()); + + // Restart NN while JN0 is still down. + cluster.restartNameNode(); + + contents = DFSTestUtil.urlGet(url); + System.out.println(contents); + assertTrue(Pattern.compile("never written").matcher(contents) + .find()); + + + } finally { + cluster.shutdown(); + } + + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java new file mode 100644 index 00000000000..41138d0eefa --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestEpochsAreUnique.java @@ -0,0 +1,144 @@ +/** + * 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.net.URI; +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.client.AsyncLogger; +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.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); + 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); + QuorumJournalManager qjm = new QuorumJournalManager( + conf, uri, FAKE_NSINFO); + try { + qjm.format(FAKE_NSINFO); + } finally { + qjm.close(); + } + + try { + // With no failures or contention, epochs should increase one-by-one + for (int i = 0; i < 5; i++) { + qjm = new QuorumJournalManager( + conf, uri, FAKE_NSINFO); + try { + qjm.createNewUniqueEpoch(); + assertEquals(i + 1, qjm.getLoggerSetForTests().getEpoch()); + } finally { + qjm.close(); + } + } + + long prevEpoch = 5; + // With some failures injected, it should still always increase, perhaps + // skipping some + for (int i = 0; i < 20; i++) { + long newEpoch = -1; + while (true) { + qjm = new QuorumJournalManager( + conf, uri, FAKE_NSINFO, new FaultyLoggerFactory()); + try { + qjm.createNewUniqueEpoch(); + newEpoch = qjm.getLoggerSetForTests().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 + } finally { + qjm.close(); + } + } + LOG.info("Created epoch " + newEpoch); + assertTrue("New epoch " + newEpoch + " should be greater than previous " + + prevEpoch, newEpoch > prevEpoch); + prevEpoch = newEpoch; + } + } finally { + cluster.shutdown(); + } + } + + private class FaultyLoggerFactory implements AsyncLogger.Factory { + @Override + public AsyncLogger createLogger(Configuration conf, NamespaceInfo nsInfo, + String journalId, InetSocketAddress addr) { + AsyncLogger ch = IPCLoggerChannel.FACTORY.createLogger( + conf, nsInfo, journalId, addr); + AsyncLogger spy = Mockito.spy(ch); + Mockito.doAnswer(new SometimesFaulty(0.10f)) + .when(spy).getJournalState(); + Mockito.doAnswer(new SometimesFaulty(0.40f)) + .when(spy).newEpoch(Mockito.anyLong()); + + return spy; + } + + } + + private class SometimesFaulty implements Answer> { + private float faultProbability; + + public SometimesFaulty(float faultProbability) { + this.faultProbability = faultProbability; + } + + @SuppressWarnings("unchecked") + @Override + public ListenableFuture answer(InvocationOnMock invocation) + throws Throwable { + if (r.nextFloat() < faultProbability) { + return Futures.immediateFailedFuture( + new IOException("Injected fault")); + } + return (ListenableFuture)invocation.callRealMethod(); + } + } + + + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestIPCLoggerChannel.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestIPCLoggerChannel.java new file mode 100644 index 00000000000..425bc507eb3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestIPCLoggerChannel.java @@ -0,0 +1,181 @@ +/** + * 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); + 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, 1, 3, FAKE_DATA).get(); + Mockito.verify(mockProxy).journal(Mockito.any(), + Mockito.eq(1L), Mockito.eq(1L), + Mockito.eq(3), Mockito.same(FAKE_DATA)); + } + + + /** + * Test that, once the queue eclipses the configure size limit, + * calls to journal more data are rejected. + */ + @Test + public void testQueueLimiting() throws Exception { + + // Block the underlying fake proxy from actually completing any calls. + DelayAnswer delayer = new DelayAnswer(LOG); + Mockito.doAnswer(delayer).when(mockProxy).journal( + Mockito.any(), + Mockito.eq(1L), Mockito.eq(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(1L, (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(1L, numToQueue + 1, 1, FAKE_DATA).get(1, TimeUnit.SECONDS); + fail("Did not fail to queue more calls after queue was full"); + } catch (ExecutionException ee) { + if (!(ee.getCause() instanceof LoggerTooFarBehindException)) { + throw ee; + } + } + + delayer.proceed(); + + // After we allow it to proceeed, it should chug through the original queue + GenericTestUtils.waitFor(new Supplier() { + @Override + public Boolean get() { + return ch.getQueuedEditsSize() == 0; + } + }, 10, 1000); + } + + /** + * Test that, if the remote node gets unsynchronized (eg some edits were + * missed or the node rebooted), the client stops sending edits until + * the next roll. Test for HDFS-3726. + */ + @Test + public void testStopSendingEditsWhenOutOfSync() throws Exception { + Mockito.doThrow(new IOException("injected error")) + .when(mockProxy).journal( + Mockito.any(), + Mockito.eq(1L), Mockito.eq(1L), + Mockito.eq(1), Mockito.same(FAKE_DATA)); + + try { + ch.sendEdits(1L, 1L, 1, FAKE_DATA).get(); + fail("Injected JOOSE did not cause sendEdits() to throw"); + } catch (ExecutionException ee) { + GenericTestUtils.assertExceptionContains("injected", ee); + } + Mockito.verify(mockProxy).journal( + Mockito.any(), + Mockito.eq(1L), Mockito.eq(1L), + Mockito.eq(1), Mockito.same(FAKE_DATA)); + + assertTrue(ch.isOutOfSync()); + + try { + ch.sendEdits(1L, 2L, 1, FAKE_DATA).get(); + fail("sendEdits() should throw until next roll"); + } catch (ExecutionException ee) { + GenericTestUtils.assertExceptionContains("disabled until next roll", + ee.getCause()); + } + + // It should have failed without even sending the edits, since it was not sync. + Mockito.verify(mockProxy, Mockito.never()).journal( + Mockito.any(), + Mockito.eq(1L), Mockito.eq(2L), + Mockito.eq(1), Mockito.same(FAKE_DATA)); + // It should have sent a heartbeat instead. + Mockito.verify(mockProxy).heartbeat( + Mockito.any()); + + // After a roll, sending new edits should not fail. + ch.startLogSegment(3L).get(); + assertFalse(ch.isOutOfSync()); + + ch.sendEdits(3L, 3L, 1, FAKE_DATA).get(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java new file mode 100644 index 00000000000..74daa58c775 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java @@ -0,0 +1,512 @@ +/** + * 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.apache.hadoop.hdfs.qjournal.QJMTestUtil.FAKE_NSINFO; +import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.JID; +import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.writeSegment; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.Closeable; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.net.InetSocketAddress; +import java.net.URISyntaxException; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.SortedSet; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.commons.logging.impl.Log4JLogger; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster; +import org.apache.hadoop.hdfs.qjournal.QJMTestUtil; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol; +import org.apache.hadoop.hdfs.qjournal.server.JournalFaultInjector; +import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream; +import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream; +import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; +import org.apache.hadoop.hdfs.util.Holder; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.ipc.ProtobufRpcEngine; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.log4j.Level; +import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.MoreExecutors; + + +public class TestQJMWithFaults { + private static final Log LOG = LogFactory.getLog( + TestQJMWithFaults.class); + + private static final String RAND_SEED_PROPERTY = + "TestQJMWithFaults.random-seed"; + + private static final int NUM_WRITER_ITERS = 500; + private static final int SEGMENTS_PER_WRITER = 2; + + private static Configuration conf = new Configuration(); + + + static { + // Don't retry connections - it just slows down the tests. + conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0); + + // Make tests run faster by avoiding fsync() + EditLogFileOutputStream.setShouldSkipFsyncForTesting(true); + } + + // Set up fault injection mock. + private static JournalFaultInjector faultInjector = + JournalFaultInjector.instance = Mockito.mock(JournalFaultInjector.class); + + /** + * Run through the creation of a log without any faults injected, + * and count how many RPCs are made to each node. This sets the + * bounds for the other test cases, so they can exhaustively explore + * the space of potential failures. + */ + private static long determineMaxIpcNumber() throws Exception { + Configuration conf = new Configuration(); + MiniJournalCluster cluster = new MiniJournalCluster.Builder(conf).build(); + QuorumJournalManager qjm = null; + long ret; + try { + qjm = createInjectableQJM(cluster); + qjm.format(FAKE_NSINFO); + doWorkload(cluster, qjm); + + SortedSet ipcCounts = Sets.newTreeSet(); + for (AsyncLogger l : qjm.getLoggerSetForTests().getLoggersForTests()) { + InvocationCountingChannel ch = (InvocationCountingChannel)l; + ch.waitForAllPendingCalls(); + ipcCounts.add(ch.getRpcCount()); + } + + // All of the loggers should have sent the same number of RPCs, since there + // were no failures. + assertEquals(1, ipcCounts.size()); + + ret = ipcCounts.first(); + LOG.info("Max IPC count = " + ret); + } finally { + IOUtils.closeStream(qjm); + cluster.shutdown(); + } + return ret; + } + + /** + * Sets up two of the nodes to each drop a single RPC, at all + * possible combinations of RPCs. This may result in the + * active writer failing to write. After this point, a new writer + * should be able to recover and continue writing without + * data loss. + */ + @Test + public void testRecoverAfterDoubleFailures() throws Exception { + final long MAX_IPC_NUMBER = determineMaxIpcNumber(); + + for (int failA = 1; failA <= MAX_IPC_NUMBER; failA++) { + for (int failB = 1; failB <= MAX_IPC_NUMBER; failB++) { + String injectionStr = "(" + failA + ", " + failB + ")"; + + LOG.info("\n\n-------------------------------------------\n" + + "Beginning test, failing at " + injectionStr + "\n" + + "-------------------------------------------\n\n"); + + MiniJournalCluster cluster = new MiniJournalCluster.Builder(conf) + .build(); + QuorumJournalManager qjm = null; + try { + qjm = createInjectableQJM(cluster); + qjm.format(FAKE_NSINFO); + List loggers = qjm.getLoggerSetForTests().getLoggersForTests(); + failIpcNumber(loggers.get(0), failA); + failIpcNumber(loggers.get(1), failB); + int lastAckedTxn = doWorkload(cluster, qjm); + + if (lastAckedTxn < 6) { + LOG.info("Failed after injecting failures at " + injectionStr + + ". This is expected since we injected a failure in the " + + "majority."); + } + qjm.close(); + qjm = null; + + // Now should be able to recover + qjm = createInjectableQJM(cluster); + long lastRecoveredTxn = QJMTestUtil.recoverAndReturnLastTxn(qjm); + assertTrue(lastRecoveredTxn >= lastAckedTxn); + + writeSegment(cluster, qjm, lastRecoveredTxn + 1, 3, true); + } catch (Throwable t) { + // Test failure! Rethrow with the test setup info so it can be + // easily triaged. + throw new RuntimeException("Test failed with injection: " + injectionStr, + t); + } finally { + cluster.shutdown(); + cluster = null; + IOUtils.closeStream(qjm); + qjm = null; + } + } + } + } + + /** + * Test case in which three JournalNodes randomly flip flop between + * up and down states every time they get an RPC. + * + * The writer keeps track of the latest ACKed edit, and on every + * recovery operation, ensures that it recovers at least to that + * point or higher. Since at any given point, a majority of JNs + * may be injecting faults, any writer operation is allowed to fail, + * so long as the exception message indicates it failed due to injected + * faults. + * + * Given a random seed, the test should be entirely deterministic. + */ + @Test + public void testRandomized() throws Exception { + long seed; + Long userSpecifiedSeed = Long.getLong(RAND_SEED_PROPERTY); + if (userSpecifiedSeed != null) { + LOG.info("Using seed specified in system property"); + seed = userSpecifiedSeed; + + // If the user specifies a seed, then we should gather all the + // IPC trace information so that debugging is easier. This makes + // the test run about 25% slower otherwise. + ((Log4JLogger)ProtobufRpcEngine.LOG).getLogger().setLevel(Level.ALL); + } else { + seed = new Random().nextLong(); + } + LOG.info("Random seed: " + seed); + + Random r = new Random(seed); + + MiniJournalCluster cluster = new MiniJournalCluster.Builder(conf) + .build(); + + // Format the cluster using a non-faulty QJM. + QuorumJournalManager qjmForInitialFormat = + createInjectableQJM(cluster); + qjmForInitialFormat.format(FAKE_NSINFO); + qjmForInitialFormat.close(); + + try { + long txid = 0; + long lastAcked = 0; + + for (int i = 0; i < NUM_WRITER_ITERS; i++) { + LOG.info("Starting writer " + i + "\n-------------------"); + + QuorumJournalManager qjm = createRandomFaultyQJM(cluster, r); + try { + long recovered; + try { + recovered = QJMTestUtil.recoverAndReturnLastTxn(qjm); + } catch (Throwable t) { + LOG.info("Failed recovery", t); + checkException(t); + continue; + } + assertTrue("Recovered only up to txnid " + recovered + + " but had gotten an ack for " + lastAcked, + recovered >= lastAcked); + + txid = recovered + 1; + + // Periodically purge old data on disk so it's easier to look + // at failure cases. + if (txid > 100 && i % 10 == 1) { + qjm.purgeLogsOlderThan(txid - 100); + } + + Holder thrown = new Holder(null); + for (int j = 0; j < SEGMENTS_PER_WRITER; j++) { + lastAcked = writeSegmentUntilCrash(cluster, qjm, txid, 4, thrown); + if (thrown.held != null) { + LOG.info("Failed write", thrown.held); + checkException(thrown.held); + break; + } + txid += 4; + } + } finally { + qjm.close(); + } + } + } finally { + cluster.shutdown(); + } + } + + private void checkException(Throwable t) { + GenericTestUtils.assertExceptionContains("Injected", t); + if (t.toString().contains("AssertionError")) { + throw new RuntimeException("Should never see AssertionError in fault test!", + t); + } + } + + private long writeSegmentUntilCrash(MiniJournalCluster cluster, + QuorumJournalManager qjm, long txid, int numTxns, Holder thrown) { + + long firstTxId = txid; + long lastAcked = txid - 1; + try { + EditLogOutputStream stm = qjm.startLogSegment(txid); + + for (int i = 0; i < numTxns; i++) { + QJMTestUtil.writeTxns(stm, txid++, 1); + lastAcked++; + } + + stm.close(); + qjm.finalizeLogSegment(firstTxId, lastAcked); + } catch (Throwable t) { + thrown.held = t; + } + return lastAcked; + } + + /** + * Run a simple workload of becoming the active writer and writing + * two log segments: 1-3 and 4-6. + */ + private static int doWorkload(MiniJournalCluster cluster, + QuorumJournalManager qjm) throws IOException { + int lastAcked = 0; + try { + qjm.recoverUnfinalizedSegments(); + writeSegment(cluster, qjm, 1, 3, true); + lastAcked = 3; + writeSegment(cluster, qjm, 4, 3, true); + lastAcked = 6; + } catch (QuorumException qe) { + LOG.info("Failed to write at txid " + lastAcked, + qe); + } + return lastAcked; + } + + /** + * Inject a failure at the given IPC number, such that the JN never + * receives the RPC. The client side sees an IOException. Future + * IPCs after this number will be received as usual. + */ + private void failIpcNumber(AsyncLogger logger, int idx) { + ((InvocationCountingChannel)logger).failIpcNumber(idx); + } + + private static class RandomFaultyChannel extends IPCLoggerChannel { + private final Random random; + private float injectionProbability = 0.1f; + private boolean isUp = true; + + public RandomFaultyChannel(Configuration conf, NamespaceInfo nsInfo, + String journalId, InetSocketAddress addr, long seed) { + super(conf, nsInfo, journalId, addr); + this.random = new Random(seed); + } + + @Override + protected QJournalProtocol createProxy() throws IOException { + QJournalProtocol realProxy = super.createProxy(); + return mockProxy( + new WrapEveryCall(realProxy) { + @Override + void beforeCall(InvocationOnMock invocation) throws Exception { + if (random.nextFloat() < injectionProbability) { + isUp = !isUp; + LOG.info("transitioned " + addr + " to " + + (isUp ? "up" : "down")); + } + + if (!isUp) { + throw new IOException("Injected - faking being down"); + } + + if (invocation.getMethod().getName().equals("acceptRecovery")) { + if (random.nextFloat() < injectionProbability) { + Mockito.doThrow(new IOException( + "Injected - faking fault before persisting paxos data")) + .when(faultInjector).beforePersistPaxosData(); + } else if (random.nextFloat() < injectionProbability) { + Mockito.doThrow(new IOException( + "Injected - faking fault after persisting paxos data")) + .when(faultInjector).afterPersistPaxosData(); + } + } + } + + @Override + public void afterCall(InvocationOnMock invocation, boolean succeeded) { + Mockito.reset(faultInjector); + } + }); + } + + @Override + protected ExecutorService createExecutor() { + return MoreExecutors.sameThreadExecutor(); + } + } + + private static class InvocationCountingChannel extends IPCLoggerChannel { + private int rpcCount = 0; + private Map> injections = Maps.newHashMap(); + + public InvocationCountingChannel(Configuration conf, NamespaceInfo nsInfo, + String journalId, InetSocketAddress addr) { + super(conf, nsInfo, journalId, addr); + } + + int getRpcCount() { + return rpcCount; + } + + void failIpcNumber(final int idx) { + Preconditions.checkArgument(idx > 0, + "id must be positive"); + inject(idx, new Callable() { + @Override + public Void call() throws Exception { + throw new IOException("injected failed IPC at " + idx); + } + }); + } + + private void inject(int beforeRpcNumber, Callable injectedCode) { + injections.put(beforeRpcNumber, injectedCode); + } + + @Override + protected QJournalProtocol createProxy() throws IOException { + final QJournalProtocol realProxy = super.createProxy(); + QJournalProtocol mock = mockProxy( + new WrapEveryCall(realProxy) { + void beforeCall(InvocationOnMock invocation) throws Exception { + rpcCount++; + String callStr = "[" + addr + "] " + + invocation.getMethod().getName() + "(" + + Joiner.on(", ").join(invocation.getArguments()) + ")"; + + Callable inject = injections.get(rpcCount); + if (inject != null) { + LOG.info("Injecting code before IPC #" + rpcCount + ": " + + callStr); + inject.call(); + } else { + LOG.info("IPC call #" + rpcCount + ": " + callStr); + } + } + }); + return mock; + } + } + + + private static QJournalProtocol mockProxy(WrapEveryCall wrapper) + throws IOException { + QJournalProtocol mock = Mockito.mock(QJournalProtocol.class, + Mockito.withSettings() + .defaultAnswer(wrapper) + .extraInterfaces(Closeable.class)); + return mock; + } + + private static abstract class WrapEveryCall implements Answer { + private final Object realObj; + WrapEveryCall(Object realObj) { + this.realObj = realObj; + } + + @SuppressWarnings("unchecked") + @Override + public T answer(InvocationOnMock invocation) throws Throwable { + // Don't want to inject an error on close() since that isn't + // actually an IPC call! + if (!Closeable.class.equals( + invocation.getMethod().getDeclaringClass())) { + beforeCall(invocation); + } + boolean success = false; + try { + T ret = (T) invocation.getMethod().invoke(realObj, + invocation.getArguments()); + success = true; + return ret; + } catch (InvocationTargetException ite) { + throw ite.getCause(); + } finally { + afterCall(invocation, success); + } + } + + abstract void beforeCall(InvocationOnMock invocation) throws Exception; + void afterCall(InvocationOnMock invocation, boolean succeeded) {} + } + + private static QuorumJournalManager createInjectableQJM(MiniJournalCluster cluster) + throws IOException, URISyntaxException { + AsyncLogger.Factory spyFactory = new AsyncLogger.Factory() { + @Override + public AsyncLogger createLogger(Configuration conf, NamespaceInfo nsInfo, + String journalId, InetSocketAddress addr) { + return new InvocationCountingChannel(conf, nsInfo, journalId, addr); + } + }; + return new QuorumJournalManager(conf, cluster.getQuorumJournalURI(JID), + FAKE_NSINFO, spyFactory); + } + + private static QuorumJournalManager createRandomFaultyQJM( + MiniJournalCluster cluster, final Random seedGenerator) + throws IOException, URISyntaxException { + + AsyncLogger.Factory spyFactory = new AsyncLogger.Factory() { + @Override + public AsyncLogger createLogger(Configuration conf, NamespaceInfo nsInfo, + String journalId, InetSocketAddress addr) { + return new RandomFaultyChannel(conf, nsInfo, journalId, addr, + seedGenerator.nextLong()); + } + }; + return new QuorumJournalManager(conf, cluster.getQuorumJournalURI(JID), + FAKE_NSINFO, spyFactory); + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java new file mode 100644 index 00000000000..2295384ea09 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.qjournal.client; + +import static org.junit.Assert.*; + +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.TimeoutException; + +import org.apache.hadoop.hdfs.qjournal.client.QuorumCall; +import org.junit.Test; + +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.SettableFuture; + +public class TestQuorumCall { + @Test(timeout=10000) + public void testQuorums() throws Exception { + Map> futures = ImmutableMap.of( + "f1", SettableFuture.create(), + "f2", SettableFuture.create(), + "f3", SettableFuture.create()); + + QuorumCall q = QuorumCall.create(futures); + assertEquals(0, q.countResponses()); + + futures.get("f1").set("first future"); + q.waitFor(1, 0, 0, 100000, "test"); // wait for 1 response + q.waitFor(0, 1, 0, 100000, "test"); // 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, "test"); // wait for 3 responses + q.waitFor(0, 2, 100, 100000, "test"); // 2 successes + + assertEquals(3, q.countResponses()); + assertEquals("f1=first future,f3=second future", + Joiner.on(",").withKeyValueSeparator("=").join( + new TreeMap(q.getResults()))); + + try { + q.waitFor(0, 4, 100, 10, "test"); + fail("Didn't time out waiting for more responses than came back"); + } catch (TimeoutException te) { + // expected + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java new file mode 100644 index 00000000000..7c2bb29d409 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManager.java @@ -0,0 +1,941 @@ +/** + * 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 static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.JID; +import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.FAKE_NSINFO; +import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.writeSegment; +import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.writeTxns; +import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.verifyEdits; +import static org.apache.hadoop.hdfs.qjournal.client.TestQuorumJournalManagerUnit.futureThrows; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.URISyntaxException; +import java.net.URL; +import java.util.List; +import java.util.concurrent.ExecutorService; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.commons.logging.impl.Log4JLogger; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster; +import org.apache.hadoop.hdfs.qjournal.QJMTestUtil; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto; +import org.apache.hadoop.hdfs.qjournal.server.JournalFaultInjector; +import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream; +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.io.IOUtils; +import org.apache.hadoop.ipc.ProtobufRpcEngine; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.log4j.Level; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.stubbing.Stubber; + +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.MoreExecutors; + +/** + * Functional tests for QuorumJournalManager. + * For true unit tests, see {@link TestQuorumJournalManagerUnit}. + */ +public class TestQuorumJournalManager { + private static final Log LOG = LogFactory.getLog( + TestQuorumJournalManager.class); + + private MiniJournalCluster cluster; + private Configuration conf; + private QuorumJournalManager qjm; + private List spies; + + static { + ((Log4JLogger)ProtobufRpcEngine.LOG).getLogger().setLevel(Level.ALL); + } + + @Before + public void setup() throws Exception { + conf = new Configuration(); + // Don't retry connections - it just slows down the tests. + conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0); + + cluster = new MiniJournalCluster.Builder(conf) + .build(); + + qjm = createSpyingQJM(); + spies = qjm.getLoggerSetForTests().getLoggersForTests(); + + qjm.format(QJMTestUtil.FAKE_NSINFO); + qjm.recoverUnfinalizedSegments(); + assertEquals(1, qjm.getLoggerSetForTests().getEpoch()); + } + + @After + public void shutdown() throws IOException { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testSingleWriter() throws Exception { + writeSegment(cluster, qjm, 1, 3, true); + + // Should be finalized + checkRecovery(cluster, 1, 3); + + // Start a new segment + writeSegment(cluster, qjm, 4, 1, true); + + // Should be finalized + checkRecovery(cluster, 4, 4); + } + + @Test + public void testFormat() throws Exception { + QuorumJournalManager qjm = new QuorumJournalManager( + conf, cluster.getQuorumJournalURI("testFormat-jid"), FAKE_NSINFO); + assertFalse(qjm.hasSomeData()); + qjm.format(FAKE_NSINFO); + assertTrue(qjm.hasSomeData()); + } + + @Test + public void testReaderWhileAnotherWrites() throws Exception { + + QuorumJournalManager readerQjm = createSpyingQJM(); + List streams = Lists.newArrayList(); + readerQjm.selectInputStreams(streams, 0, false); + assertEquals(0, streams.size()); + writeSegment(cluster, qjm, 1, 3, true); + + readerQjm.selectInputStreams(streams, 0, false); + try { + assertEquals(1, streams.size()); + // Validate the actual stream contents. + EditLogInputStream stream = streams.get(0); + assertEquals(1, stream.getFirstTxId()); + assertEquals(3, stream.getLastTxId()); + + verifyEdits(streams, 1, 3); + assertNull(stream.readOp()); + } finally { + IOUtils.cleanup(LOG, streams.toArray(new Closeable[0])); + streams.clear(); + } + + // Ensure correct results when there is a stream in-progress, but we don't + // ask for in-progress. + writeSegment(cluster, qjm, 4, 3, false); + readerQjm.selectInputStreams(streams, 0, false); + try { + assertEquals(1, streams.size()); + EditLogInputStream stream = streams.get(0); + assertEquals(1, stream.getFirstTxId()); + assertEquals(3, stream.getLastTxId()); + verifyEdits(streams, 1, 3); + } finally { + IOUtils.cleanup(LOG, streams.toArray(new Closeable[0])); + streams.clear(); + } + + // TODO: check results for selectInputStreams with inProgressOK = true. + // This doesn't currently work, due to a bug where RedundantEditInputStream + // throws an exception if there are any unvalidated in-progress edits in the list! + // But, it shouldn't be necessary for current use cases. + + qjm.finalizeLogSegment(4, 6); + readerQjm.selectInputStreams(streams, 0, false); + try { + assertEquals(2, streams.size()); + assertEquals(4, streams.get(1).getFirstTxId()); + assertEquals(6, streams.get(1).getLastTxId()); + + verifyEdits(streams, 1, 6); + } finally { + IOUtils.cleanup(LOG, streams.toArray(new Closeable[0])); + streams.clear(); + } + } + + /** + * Regression test for HDFS-3725. One of the journal nodes is down + * during the writing of one segment, then comes back up later to + * take part in a later segment. Thus, its local edits are + * not a contiguous sequence. This should be handled correctly. + */ + @Test + public void testOneJNMissingSegments() throws Exception { + writeSegment(cluster, qjm, 1, 3, true); + waitForAllPendingCalls(qjm.getLoggerSetForTests()); + cluster.getJournalNode(0).stopAndJoin(0); + writeSegment(cluster, qjm, 4, 3, true); + waitForAllPendingCalls(qjm.getLoggerSetForTests()); + cluster.restartJournalNode(0); + writeSegment(cluster, qjm, 7, 3, true); + waitForAllPendingCalls(qjm.getLoggerSetForTests()); + cluster.getJournalNode(1).stopAndJoin(0); + + QuorumJournalManager readerQjm = createSpyingQJM(); + List streams = Lists.newArrayList(); + try { + readerQjm.selectInputStreams(streams, 1, false); + verifyEdits(streams, 1, 9); + } finally { + IOUtils.cleanup(LOG, streams.toArray(new Closeable[0])); + readerQjm.close(); + } + } + + /** + * Regression test for HDFS-3891: selectInputStreams should throw + * an exception when a majority of journalnodes have crashed. + */ + @Test + public void testSelectInputStreamsMajorityDown() throws Exception { + // Shut down all of the JNs. + cluster.shutdown(); + + List streams = Lists.newArrayList(); + try { + qjm.selectInputStreams(streams, 0, false); + fail("Did not throw IOE"); + } catch (QuorumException ioe) { + GenericTestUtils.assertExceptionContains( + "Got too many exceptions", ioe); + assertTrue(streams.isEmpty()); + } + } + + /** + * Test the case where the NN crashes after starting a new segment + * on all nodes, but before writing the first transaction to it. + */ + @Test + public void testCrashAtBeginningOfSegment() throws Exception { + writeSegment(cluster, qjm, 1, 3, true); + waitForAllPendingCalls(qjm.getLoggerSetForTests()); + + EditLogOutputStream stm = qjm.startLogSegment(4); + try { + waitForAllPendingCalls(qjm.getLoggerSetForTests()); + } finally { + stm.abort(); + } + + + // Make a new QJM + qjm = new QuorumJournalManager( + conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO); + qjm.recoverUnfinalizedSegments(); + checkRecovery(cluster, 1, 3); + + writeSegment(cluster, qjm, 4, 3, true); + } + + @Test + public void testOutOfSyncAtBeginningOfSegment0() throws Exception { + doTestOutOfSyncAtBeginningOfSegment(0); + } + + @Test + public void testOutOfSyncAtBeginningOfSegment1() throws Exception { + doTestOutOfSyncAtBeginningOfSegment(1); + } + + @Test + public void testOutOfSyncAtBeginningOfSegment2() throws Exception { + doTestOutOfSyncAtBeginningOfSegment(2); + } + + /** + * Test the case where, at the beginning of a segment, transactions + * have been written to one JN but not others. + */ + public void doTestOutOfSyncAtBeginningOfSegment(int nodeWithOneTxn) + throws Exception { + + int nodeWithEmptySegment = (nodeWithOneTxn + 1) % 3; + int nodeMissingSegment = (nodeWithOneTxn + 2) % 3; + + writeSegment(cluster, qjm, 1, 3, true); + waitForAllPendingCalls(qjm.getLoggerSetForTests()); + cluster.getJournalNode(nodeMissingSegment).stopAndJoin(0); + + // Open segment on 2/3 nodes + EditLogOutputStream stm = qjm.startLogSegment(4); + try { + waitForAllPendingCalls(qjm.getLoggerSetForTests()); + + // Write transactions to only 1/3 nodes + failLoggerAtTxn(spies.get(nodeWithEmptySegment), 4); + try { + writeTxns(stm, 4, 1); + fail("Did not fail even though 2/3 failed"); + } catch (QuorumException qe) { + GenericTestUtils.assertExceptionContains("mock failure", qe); + } + } finally { + stm.abort(); + } + + // Bring back the down JN. + cluster.restartJournalNode(nodeMissingSegment); + + // Make a new QJM. At this point, the state is as follows: + // A: nodeWithEmptySegment: 1-3 finalized, 4_inprogress (empty) + // B: nodeWithOneTxn: 1-3 finalized, 4_inprogress (1 txn) + // C: nodeMissingSegment: 1-3 finalized + GenericTestUtils.assertGlobEquals( + cluster.getCurrentDir(nodeWithEmptySegment, JID), + "edits_.*", + NNStorage.getFinalizedEditsFileName(1, 3), + NNStorage.getInProgressEditsFileName(4)); + GenericTestUtils.assertGlobEquals( + cluster.getCurrentDir(nodeWithOneTxn, JID), + "edits_.*", + NNStorage.getFinalizedEditsFileName(1, 3), + NNStorage.getInProgressEditsFileName(4)); + GenericTestUtils.assertGlobEquals( + cluster.getCurrentDir(nodeMissingSegment, JID), + "edits_.*", + NNStorage.getFinalizedEditsFileName(1, 3)); + + + // Stop one of the nodes. Since we run this test three + // times, rotating the roles of the nodes, we'll test + // all the permutations. + cluster.getJournalNode(2).stopAndJoin(0); + + qjm = createSpyingQJM(); + qjm.recoverUnfinalizedSegments(); + + if (nodeWithOneTxn == 0 || + nodeWithOneTxn == 1) { + // If the node that had the transaction committed was one of the nodes + // that responded during recovery, then we should have recovered txid + // 4. + checkRecovery(cluster, 4, 4); + writeSegment(cluster, qjm, 5, 3, true); + } else { + // Otherwise, we should have recovered only 1-3 and should be able to + // start a segment at 4. + checkRecovery(cluster, 1, 3); + writeSegment(cluster, qjm, 4, 3, true); + } + } + + + /** + * 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(cluster, qjm, 1, 3, false); + QJMTestUtil.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 { + setupLoggers345(); + + QJMTestUtil.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.anyLong(), + Mockito.eq(txid), Mockito.eq(1), Mockito.any()); + } + + /** + * Test the case where one of the loggers misses a finalizeLogSegment() + * call, and then misses the next startLogSegment() call before coming + * back to life. + * + * Previously, this caused it to keep on writing to the old log segment, + * such that one logger had eg edits_1-10 while the others had edits_1-5 and + * edits_6-10. This caused recovery to fail in certain cases. + */ + @Test + public void testMissFinalizeAndNextStart() throws Exception { + + // Logger 0: miss finalize(1-3) and start(4) + futureThrows(new IOException("injected")).when(spies.get(0)) + .finalizeLogSegment(Mockito.eq(1L), Mockito.eq(3L)); + futureThrows(new IOException("injected")).when(spies.get(0)) + .startLogSegment(Mockito.eq(4L)); + + // Logger 1: fail at txn id 4 + failLoggerAtTxn(spies.get(1), 4L); + + writeSegment(cluster, qjm, 1, 3, true); + EditLogOutputStream stm = qjm.startLogSegment(4); + try { + writeTxns(stm, 4, 1); + fail("Did not fail to write"); + } catch (QuorumException qe) { + // Should fail, because logger 1 had an injected fault and + // logger 0 should detect writer out of sync + GenericTestUtils.assertExceptionContains("Writer out of sync", + qe); + } finally { + stm.abort(); + qjm.close(); + } + + // State: + // Logger 0: 1-3 in-progress (since it missed finalize) + // Logger 1: 1-3 finalized + // Logger 2: 1-3 finalized, 4 in-progress with one txn + + // Shut down logger 2 so it doesn't participate in recovery + cluster.getJournalNode(2).stopAndJoin(0); + + qjm = createSpyingQJM(); + long recovered = QJMTestUtil.recoverAndReturnLastTxn(qjm); + assertEquals(3L, recovered); + } + + /** + * 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 { + setupLoggers345(); + + // 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); + } + + /** + * Set up the loggers into the following state: + * - JN0: edits 1-3 in progress + * - JN1: edits 1-4 in progress + * - JN2: edits 1-5 in progress + * + * None of the loggers have any associated paxos info. + */ + private void setupLoggers345() 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); + } + } + + /** + * Set up the following tricky edge case state which is used by + * multiple tests: + * + * Initial writer: + * - Writing to 3 JNs: JN0, JN1, JN2: + * - A log segment with txnid 1 through 100 succeeds. + * - The first transaction in the next segment only goes to JN0 + * before the writer crashes (eg it is partitioned) + * + * Recovery by another writer: + * - The new NN starts recovery and talks to all three. Thus, it sees + * that the newest log segment which needs recovery is 101. + * - It sends the prepareRecovery(101) call, and decides that the + * recovery length for 101 is only the 1 transaction. + * - It sends acceptRecovery(101-101) to only JN0, before crashing + * + * This yields the following state: + * - JN0: 1-100 finalized, 101_inprogress, accepted recovery: 101-101 + * - JN1: 1-100 finalized, 101_inprogress.empty + * - JN2: 1-100 finalized, 101_inprogress.empty + * (the .empty files got moved aside during recovery) + * @throws Exception + */ + private void setupEdgeCaseOneJnHasSegmentWithAcceptedRecovery() throws Exception { + // Log segment with txns 1-100 succeeds + writeSegment(cluster, qjm, 1, 100, true); + + // startLogSegment only makes it to one of the three nodes + failLoggerAtTxn(spies.get(1), 101); + failLoggerAtTxn(spies.get(2), 101); + + try { + writeSegment(cluster, qjm, 101, 1, true); + fail("Should have failed"); + } catch (QuorumException qe) { + GenericTestUtils.assertExceptionContains("mock failure", qe); + } finally { + qjm.close(); + } + + // Recovery 1: + // make acceptRecovery() only make it to the node which has txid 101 + // this should fail because only 1/3 accepted the recovery + qjm = createSpyingQJM(); + spies = qjm.getLoggerSetForTests().getLoggersForTests(); + futureThrows(new IOException("mock failure")).when(spies.get(1)) + .acceptRecovery(Mockito.any(), Mockito.any()); + futureThrows(new IOException("mock failure")).when(spies.get(2)) + .acceptRecovery(Mockito.any(), Mockito.any()); + + try { + qjm.recoverUnfinalizedSegments(); + fail("Should have failed to recover"); + } catch (QuorumException qe) { + GenericTestUtils.assertExceptionContains("mock failure", qe); + } finally { + qjm.close(); + } + + // Check that we have entered the expected state as described in the + // method javadoc. + GenericTestUtils.assertGlobEquals(cluster.getCurrentDir(0, JID), + "edits_.*", + NNStorage.getFinalizedEditsFileName(1, 100), + NNStorage.getInProgressEditsFileName(101)); + GenericTestUtils.assertGlobEquals(cluster.getCurrentDir(1, JID), + "edits_.*", + NNStorage.getFinalizedEditsFileName(1, 100), + NNStorage.getInProgressEditsFileName(101) + ".empty"); + GenericTestUtils.assertGlobEquals(cluster.getCurrentDir(2, JID), + "edits_.*", + NNStorage.getFinalizedEditsFileName(1, 100), + NNStorage.getInProgressEditsFileName(101) + ".empty"); + + File paxos0 = new File(cluster.getCurrentDir(0, JID), "paxos"); + File paxos1 = new File(cluster.getCurrentDir(1, JID), "paxos"); + File paxos2 = new File(cluster.getCurrentDir(2, JID), "paxos"); + + GenericTestUtils.assertGlobEquals(paxos0, ".*", "101"); + GenericTestUtils.assertGlobEquals(paxos1, ".*"); + GenericTestUtils.assertGlobEquals(paxos2, ".*"); + } + + /** + * Test an edge case discovered by randomized testing. + * + * Starts with the edge case state set up by + * {@link #setupEdgeCaseOneJnHasSegmentWithAcceptedRecovery()} + * + * Recovery 2: + * - New NN starts recovery and only talks to JN1 and JN2. JN0 has + * crashed. Since they have no logs open, they say they don't need + * recovery. + * - Starts writing segment 101, and writes 50 transactions before crashing. + * + * Recovery 3: + * - JN0 has come back to life. + * - New NN starts recovery and talks to all three. All three have + * segments open from txid 101, so it calls prepareRecovery(101) + * - JN0 has an already-accepted value for segment 101, so it replies + * "you should recover 101-101" + * - Former incorrect behavior: NN truncates logs to txid 101 even though + * it should have recovered through 150. + * + * In this case, even though there is an accepted recovery decision, + * the newer log segments should take precedence, since they were written + * in a newer epoch than the recorded decision. + */ + @Test + public void testNewerVersionOfSegmentWins() throws Exception { + setupEdgeCaseOneJnHasSegmentWithAcceptedRecovery(); + + // Now start writing again without JN0 present: + cluster.getJournalNode(0).stopAndJoin(0); + + qjm = createSpyingQJM(); + try { + assertEquals(100, QJMTestUtil.recoverAndReturnLastTxn(qjm)); + + // Write segment but do not finalize + writeSegment(cluster, qjm, 101, 50, false); + } finally { + qjm.close(); + } + + // Now try to recover a new writer, with JN0 present, + // and ensure that all of the above-written transactions are recovered. + cluster.restartJournalNode(0); + qjm = createSpyingQJM(); + try { + assertEquals(150, QJMTestUtil.recoverAndReturnLastTxn(qjm)); + } finally { + qjm.close(); + } + } + + /** + * Test another edge case discovered by randomized testing. + * + * Starts with the edge case state set up by + * {@link #setupEdgeCaseOneJnHasSegmentWithAcceptedRecovery()} + * + * Recovery 2: + * - New NN starts recovery and only talks to JN1 and JN2. JN0 has + * crashed. Since they have no logs open, they say they don't need + * recovery. + * - Before writing any transactions, JN0 comes back to life and + * JN1 crashes. + * - Starts writing segment 101, and writes 50 transactions before crashing. + * + * Recovery 3: + * - JN1 has come back to life. JN2 crashes. + * - New NN starts recovery and talks to all three. All three have + * segments open from txid 101, so it calls prepareRecovery(101) + * - JN0 has an already-accepted value for segment 101, so it replies + * "you should recover 101-101" + * - Former incorrect behavior: NN truncates logs to txid 101 even though + * it should have recovered through 150. + * + * In this case, even though there is an accepted recovery decision, + * the newer log segments should take precedence, since they were written + * in a newer epoch than the recorded decision. + */ + @Test + public void testNewerVersionOfSegmentWins2() throws Exception { + setupEdgeCaseOneJnHasSegmentWithAcceptedRecovery(); + + // Recover without JN0 present. + cluster.getJournalNode(0).stopAndJoin(0); + + qjm = createSpyingQJM(); + try { + assertEquals(100, QJMTestUtil.recoverAndReturnLastTxn(qjm)); + + // After recovery, JN0 comes back to life and JN1 crashes. + cluster.restartJournalNode(0); + cluster.getJournalNode(1).stopAndJoin(0); + + // Write segment but do not finalize + writeSegment(cluster, qjm, 101, 50, false); + } finally { + qjm.close(); + } + + // State: + // JN0: 1-100 finalized, 101_inprogress (txns up to 150) + // Previously, JN0 had an accepted recovery 101-101 from an earlier recovery + // attempt. + // JN1: 1-100 finalized + // JN2: 1-100 finalized, 101_inprogress (txns up to 150) + + // We need to test that the accepted recovery 101-101 on JN0 doesn't + // end up truncating the log back to 101. + + cluster.restartJournalNode(1); + cluster.getJournalNode(2).stopAndJoin(0); + + qjm = createSpyingQJM(); + try { + assertEquals(150, QJMTestUtil.recoverAndReturnLastTxn(qjm)); + } finally { + qjm.close(); + } + } + + @Test(timeout=20000) + public void testCrashBetweenSyncLogAndPersistPaxosData() throws Exception { + JournalFaultInjector faultInjector = + JournalFaultInjector.instance = Mockito.mock(JournalFaultInjector.class); + + setupLoggers345(); + + // Run recovery where the client only talks to JN0, JN1, such that it + // decides that the correct length is through txid 4. + // Only allow it to call acceptRecovery() on JN0. + qjm = createSpyingQJM(); + spies = qjm.getLoggerSetForTests().getLoggersForTests(); + cluster.getJournalNode(2).stopAndJoin(0); + injectIOE().when(spies.get(1)).acceptRecovery( + Mockito.any(), Mockito.any()); + + tryRecoveryExpectingFailure(); + + cluster.restartJournalNode(2); + + // State at this point: + // JN0: edit log for 1-4, paxos recovery data for txid 4 + // JN1: edit log for 1-4, + // JN2: edit log for 1-5 + + // Run recovery again, but don't allow JN0 to respond to the + // prepareRecovery() call. This will cause recovery to decide + // on txid 5. + // Additionally, crash all of the nodes before they persist + // any new paxos data. + qjm = createSpyingQJM(); + spies = qjm.getLoggerSetForTests().getLoggersForTests(); + injectIOE().when(spies.get(0)).prepareRecovery(Mockito.eq(1L)); + + Mockito.doThrow(new IOException("Injected")).when(faultInjector) + .beforePersistPaxosData(); + tryRecoveryExpectingFailure(); + Mockito.reset(faultInjector); + + // State at this point: + // JN0: edit log for 1-5, paxos recovery data for txid 4 + // !!! This is the interesting bit, above. The on-disk data and the + // paxos data don't match up! + // JN1: edit log for 1-5, + // JN2: edit log for 1-5, + + // Now, stop JN2, and see if we can still start up even though + // JN0 is in a strange state where its log data is actually newer + // than its accepted Paxos state. + + cluster.getJournalNode(2).stopAndJoin(0); + + qjm = createSpyingQJM(); + try { + long recovered = QJMTestUtil.recoverAndReturnLastTxn(qjm); + assertTrue(recovered >= 4); // 4 was committed to a quorum + } finally { + qjm.close(); + } + } + + private void tryRecoveryExpectingFailure() throws IOException { + try { + QJMTestUtil.recoverAndReturnLastTxn(qjm); + fail("Expected to fail recovery"); + } catch (QuorumException qe) { + GenericTestUtils.assertExceptionContains("Injected", qe); + } finally { + qjm.close(); + } + + } + + private Stubber injectIOE() { + return futureThrows(new IOException("Injected")); + } + + @Test + public void testPurgeLogs() throws Exception { + for (int txid = 1; txid <= 5; txid++) { + writeSegment(cluster, qjm, txid, 1, true); + } + File curDir = cluster.getCurrentDir(0, JID); + GenericTestUtils.assertGlobEquals(curDir, "edits_.*", + NNStorage.getFinalizedEditsFileName(1, 1), + NNStorage.getFinalizedEditsFileName(2, 2), + NNStorage.getFinalizedEditsFileName(3, 3), + NNStorage.getFinalizedEditsFileName(4, 4), + NNStorage.getFinalizedEditsFileName(5, 5)); + File paxosDir = new File(curDir, "paxos"); + GenericTestUtils.assertExists(paxosDir); + + // Create new files in the paxos directory, which should get purged too. + assertTrue(new File(paxosDir, "1").createNewFile()); + assertTrue(new File(paxosDir, "3").createNewFile()); + + GenericTestUtils.assertGlobEquals(paxosDir, "\\d+", + "1", "3"); + + // Create some temporary files of the sort that are used during recovery. + assertTrue(new File(curDir, + "edits_inprogress_0000000000000000001.epoch=140").createNewFile()); + assertTrue(new File(curDir, + "edits_inprogress_0000000000000000002.empty").createNewFile()); + + qjm.purgeLogsOlderThan(3); + + // Log purging is asynchronous, so we have to wait for the calls + // to be sent and respond before verifying. + waitForAllPendingCalls(qjm.getLoggerSetForTests()); + + // Older edits should be purged + GenericTestUtils.assertGlobEquals(curDir, "edits_.*", + NNStorage.getFinalizedEditsFileName(3, 3), + NNStorage.getFinalizedEditsFileName(4, 4), + NNStorage.getFinalizedEditsFileName(5, 5)); + + // Older paxos files should be purged + GenericTestUtils.assertGlobEquals(paxosDir, "\\d+", + "3"); + } + + @Test + public void testToString() throws Exception { + GenericTestUtils.assertMatches( + qjm.toString(), + "QJM to \\[127.0.0.1:\\d+, 127.0.0.1:\\d+, 127.0.0.1:\\d+\\]"); + } + + + private QuorumJournalManager createSpyingQJM() + throws IOException, URISyntaxException { + AsyncLogger.Factory spyFactory = new AsyncLogger.Factory() { + @Override + public AsyncLogger createLogger(Configuration conf, NamespaceInfo nsInfo, + String journalId, InetSocketAddress addr) { + AsyncLogger logger = new IPCLoggerChannel(conf, nsInfo, journalId, addr) { + protected ExecutorService createExecutor() { + // Don't parallelize calls to the quorum in the tests. + // This makes the tests more deterministic. + return MoreExecutors.sameThreadExecutor(); + } + }; + + return Mockito.spy(logger); + } + }; + return new QuorumJournalManager( + conf, cluster.getQuorumJournalURI(JID), FAKE_NSINFO, spyFactory); + } + + private static void waitForAllPendingCalls(AsyncLoggerSet als) + throws InterruptedException { + for (AsyncLogger l : als.getLoggersForTests()) { + IPCLoggerChannel ch = (IPCLoggerChannel)l; + ch.waitForAllPendingCalls(); + } + } + + private void checkRecovery(MiniJournalCluster cluster, + long segmentTxId, long expectedEndTxId) + throws IOException { + int numFinalized = 0; + for (int i = 0; i < cluster.getNumNodes(); i++) { + File logDir = cluster.getCurrentDir(i, JID); + EditLogFile elf = FileJournalManager.getLogFile(logDir, segmentTxId); + if (elf == null) { + continue; + } + if (!elf.isInProgress()) { + numFinalized++; + if (elf.getLastTxId() != expectedEndTxId) { + fail("File " + elf + " finalized to wrong txid, expected " + + expectedEndTxId); + } + } + } + + if (numFinalized < cluster.getQuorumSize()) { + fail("Did not find a quorum of finalized logs starting at " + + segmentTxId); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java new file mode 100644 index 00000000000..295eab1986c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumJournalManagerUnit.java @@ -0,0 +1,200 @@ +/** + * 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.anyLong; +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.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; + +import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.writeOp; + +/** + * 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); + + private Configuration conf = new Configuration(); + private List spyLoggers; + private QuorumJournalManager qjm; + + @Before + public void setup() throws Exception { + spyLoggers = ImmutableList.of( + mockLogger(), + mockLogger(), + mockLogger()); + + qjm = new QuorumJournalManager(conf, new URI("qjournal://host/jid"), FAKE_NSINFO) { + @Override + protected List createLoggers(AsyncLogger.Factory factory) { + 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()); + + futureReturns(null).when(logger).format(Mockito.any()); + } + + qjm.recoverUnfinalizedSegments(); + } + + private AsyncLogger mockLogger() { + return Mockito.mock(AsyncLogger.class); + } + + static Stubber futureReturns(V value) { + ListenableFuture ret = Futures.immediateFuture(value); + return Mockito.doReturn(ret); + } + + static Stubber futureThrows(Throwable t) { + ListenableFuture ret = Futures.immediateFailedFuture(t); + return Mockito.doReturn(ret); + } + + + @Test + public void testAllLoggersStartOk() throws Exception { + futureReturns(null).when(spyLoggers.get(0)).startLogSegment(Mockito.anyLong()); + futureReturns(null).when(spyLoggers.get(1)).startLogSegment(Mockito.anyLong()); + futureReturns(null).when(spyLoggers.get(2)).startLogSegment(Mockito.anyLong()); + qjm.startLogSegment(1); + } + + @Test + public void testQuorumOfLoggersStartOk() throws Exception { + futureReturns(null).when(spyLoggers.get(0)).startLogSegment(Mockito.anyLong()); + futureReturns(null).when(spyLoggers.get(1)).startLogSegment(Mockito.anyLong()); + futureThrows(new IOException("logger failed")) + .when(spyLoggers.get(2)).startLogSegment(Mockito.anyLong()); + qjm.startLogSegment(1); + } + + @Test + public void testQuorumOfLoggersFail() throws Exception { + futureReturns(null).when(spyLoggers.get(0)).startLogSegment(Mockito.anyLong()); + futureThrows(new IOException("logger failed")) + .when(spyLoggers.get(1)).startLogSegment(Mockito.anyLong()); + futureThrows(new IOException("logger failed")) + .when(spyLoggers.get(2)).startLogSegment(Mockito.anyLong()); + try { + qjm.startLogSegment(1); + fail("Did not throw when quorum failed"); + } catch (QuorumException qe) { + GenericTestUtils.assertExceptionContains("logger failed", qe); + } + } + + @Test + public void testWriteEdits() throws Exception { + EditLogOutputStream stm = createLogSegment(); + writeOp(stm, 1); + writeOp(stm, 2); + + stm.setReadyToFlush(); + writeOp(stm, 3); + + // The flush should log txn 1-2 + futureReturns(null).when(spyLoggers.get(0)).sendEdits( + anyLong(), eq(1L), eq(2), Mockito.any()); + futureReturns(null).when(spyLoggers.get(1)).sendEdits( + anyLong(), eq(1L), eq(2), Mockito.any()); + futureReturns(null).when(spyLoggers.get(2)).sendEdits( + anyLong(), eq(1L), eq(2), Mockito.any()); + stm.flush(); + + // Another flush should now log txn #3 + stm.setReadyToFlush(); + futureReturns(null).when(spyLoggers.get(0)).sendEdits( + anyLong(), eq(3L), eq(1), Mockito.any()); + futureReturns(null).when(spyLoggers.get(1)).sendEdits( + anyLong(), eq(3L), eq(1), Mockito.any()); + futureReturns(null).when(spyLoggers.get(2)).sendEdits( + anyLong(), eq(3L), eq(1), Mockito.any()); + stm.flush(); + } + + @Test + public void testWriteEditsOneSlow() throws Exception { + EditLogOutputStream stm = createLogSegment(); + writeOp(stm, 1); + stm.setReadyToFlush(); + + // Make the first two logs respond immediately + futureReturns(null).when(spyLoggers.get(0)).sendEdits( + anyLong(), eq(1L), eq(1), Mockito.any()); + futureReturns(null).when(spyLoggers.get(1)).sendEdits( + anyLong(), eq(1L), eq(1), Mockito.any()); + + // And the third log not respond + SettableFuture slowLog = SettableFuture.create(); + Mockito.doReturn(slowLog).when(spyLoggers.get(2)).sendEdits( + anyLong(), eq(1L), eq(1), Mockito.any()); + stm.flush(); + + Mockito.verify(spyLoggers.get(0)).setCommittedTxId(1L); + } + + 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; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestSegmentRecoveryComparator.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestSegmentRecoveryComparator.java new file mode 100644 index 00000000000..ecd6a2cd464 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestSegmentRecoveryComparator.java @@ -0,0 +1,94 @@ +/** + * 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.Map.Entry; + +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto; +import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto; +import org.junit.Test; +import org.mockito.Mockito; + +import com.google.common.collect.Maps; + +import static org.apache.hadoop.hdfs.qjournal.client.SegmentRecoveryComparator.INSTANCE; + +public class TestSegmentRecoveryComparator { + + private static Map.Entry makeEntry( + PrepareRecoveryResponseProto proto) { + return Maps.immutableEntry(Mockito.mock(AsyncLogger.class), proto); + } + + @Test + public void testComparisons() { + Entry INPROGRESS_1_3 = + makeEntry(PrepareRecoveryResponseProto.newBuilder() + .setSegmentState(SegmentStateProto.newBuilder() + .setStartTxId(1L) + .setEndTxId(3L) + .setIsInProgress(true)) + .setLastWriterEpoch(0L) + .build()); + Entry INPROGRESS_1_4 = + makeEntry(PrepareRecoveryResponseProto.newBuilder() + .setSegmentState(SegmentStateProto.newBuilder() + .setStartTxId(1L) + .setEndTxId(4L) + .setIsInProgress(true)) + .setLastWriterEpoch(0L) + .build()); + Entry INPROGRESS_1_4_ACCEPTED = + makeEntry(PrepareRecoveryResponseProto.newBuilder() + .setSegmentState(SegmentStateProto.newBuilder() + .setStartTxId(1L) + .setEndTxId(4L) + .setIsInProgress(true)) + .setLastWriterEpoch(0L) + .setAcceptedInEpoch(1L) + .build()); + + Entry FINALIZED_1_3 = + makeEntry(PrepareRecoveryResponseProto.newBuilder() + .setSegmentState(SegmentStateProto.newBuilder() + .setStartTxId(1L) + .setEndTxId(3L) + .setIsInProgress(false)) + .setLastWriterEpoch(0L) + .build()); + + // Should compare equal to itself + assertEquals(0, INSTANCE.compare(INPROGRESS_1_3, INPROGRESS_1_3)); + + // Longer log wins. + assertEquals(-1, INSTANCE.compare(INPROGRESS_1_3, INPROGRESS_1_4)); + assertEquals(1, INSTANCE.compare(INPROGRESS_1_4, INPROGRESS_1_3)); + + // Finalized log wins even over a longer in-progress + assertEquals(-1, INSTANCE.compare(INPROGRESS_1_4, FINALIZED_1_3)); + assertEquals(1, INSTANCE.compare(FINALIZED_1_3, INPROGRESS_1_4)); + + // Finalized log wins even if the in-progress one has an accepted + // recovery proposal. + assertEquals(-1, INSTANCE.compare(INPROGRESS_1_4_ACCEPTED, FINALIZED_1_3)); + assertEquals(1, INSTANCE.compare(FINALIZED_1_3, INPROGRESS_1_4_ACCEPTED)); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java new file mode 100644 index 00000000000..c9db35faca1 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.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.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.JournalOutOfSyncException; +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.io.IOUtils; +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); + private static final NamespaceInfo FAKE_NSINFO_2 = new NamespaceInfo( + 6789, "mycluster", "my-bp", 0L); + + 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, JID, mockErrorReporter); + journal.format(FAKE_NSINFO); + } + + @After + public void verifyNoStorageErrors() throws Exception{ + Mockito.verify(mockErrorReporter, Mockito.never()) + .reportErrorOnFile(Mockito.any()); + } + + @After + public void cleanup() { + IOUtils.closeStream(journal); + } + + @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(makeRI(1), 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(makeRI(1), 12345L, 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 testMaintainCommittedTxId() throws Exception { + journal.newEpoch(FAKE_NSINFO, 1); + journal.startLogSegment(makeRI(1), 1); + // Send txids 1-3, with a request indicating only 0 committed + journal.journal(new RequestInfo(JID, 1, 2, 0), 1, 1, 3, + QJMTestUtil.createTxnData(1, 3)); + assertEquals(0, journal.getCommittedTxnIdForTests()); + + // Send 4-6, with request indicating that through 3 is committed. + journal.journal(new RequestInfo(JID, 1, 3, 3), 1, 4, 3, + QJMTestUtil.createTxnData(4, 6)); + assertEquals(3, journal.getCommittedTxnIdForTests()); + } + + @Test + public void testRestartJournal() throws Exception { + journal.newEpoch(FAKE_NSINFO, 1); + journal.startLogSegment(makeRI(1), 1); + journal.journal(makeRI(2), 1, 1, 2, + QJMTestUtil.createTxnData(1, 2)); + // Don't finalize. + + String storageString = journal.getStorage().toColonSeparatedString(); + System.err.println("storage string: " + storageString); + journal.close(); // close to unlock the storage dir + + // Now re-instantiate, make sure history is still there + journal = new Journal(TEST_LOG_DIR, JID, mockErrorReporter); + + // The storage info should be read, even if no writer has taken over. + assertEquals(storageString, + journal.getStorage().toColonSeparatedString()); + + assertEquals(1, journal.getLastPromisedEpoch()); + NewEpochResponseProtoOrBuilder newEpoch = journal.newEpoch(FAKE_NSINFO, 2); + assertEquals(1, newEpoch.getLastSegmentTxId()); + } + + @Test + public void testFormatResetsCachedValues() throws Exception { + journal.newEpoch(FAKE_NSINFO, 12345L); + journal.startLogSegment(new RequestInfo(JID, 12345L, 1L, 0L), 1L); + + assertEquals(12345L, journal.getLastPromisedEpoch()); + assertEquals(12345L, journal.getLastWriterEpoch()); + assertTrue(journal.isFormatted()); + + journal.format(FAKE_NSINFO_2); + + assertEquals(0, journal.getLastPromisedEpoch()); + assertEquals(0, journal.getLastWriterEpoch()); + assertTrue(journal.isFormatted()); + } + + /** + * Test that, if the writer crashes at the very beginning of a segment, + * before any transactions are written, that the next newEpoch() call + * returns the prior segment txid as its most recent segment. + */ + @Test + public void testNewEpochAtBeginningOfSegment() throws Exception { + journal.newEpoch(FAKE_NSINFO, 1); + journal.startLogSegment(makeRI(1), 1); + journal.journal(makeRI(2), 1, 1, 2, + QJMTestUtil.createTxnData(1, 2)); + journal.finalizeLogSegment(makeRI(3), 1, 2); + journal.startLogSegment(makeRI(4), 3); + NewEpochResponseProto resp = journal.newEpoch(FAKE_NSINFO, 2); + assertEquals(1, resp.getLastSegmentTxId()); + } + + @Test + public void testJournalLocking() throws Exception { + Assume.assumeTrue(journal.getStorage().getStorageDir(0).isLockSupported()); + StorageDirectory sd = journal.getStorage().getStorageDir(0); + File lockFile = new File(sd.getRoot(), Storage.STORAGE_FILE_LOCK); + + // Journal should be locked, since the format() call locks it. + GenericTestUtils.assertExists(lockFile); + + journal.newEpoch(FAKE_NSINFO, 1); + try { + new Journal(TEST_LOG_DIR, JID, mockErrorReporter); + 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. + // Hence, should be able to create a new Journal in the same dir. + Journal journal2 = new Journal(TEST_LOG_DIR, JID, mockErrorReporter); + journal2.newEpoch(FAKE_NSINFO, 2); + } + + /** + * Test finalizing a segment after some batch of edits were missed. + * This should fail, since we validate the log before finalization. + */ + @Test + public void testFinalizeWhenEditsAreMissed() throws Exception { + journal.newEpoch(FAKE_NSINFO, 1); + journal.startLogSegment(makeRI(1), 1); + journal.journal(makeRI(2), 1, 1, 3, + QJMTestUtil.createTxnData(1, 3)); + + // Try to finalize up to txn 6, even though we only wrote up to txn 3. + try { + journal.finalizeLogSegment(makeRI(3), 1, 6); + fail("did not fail to finalize"); + } catch (JournalOutOfSyncException e) { + GenericTestUtils.assertExceptionContains( + "but only written up to txid 3", e); + } + + // Check that, even if we re-construct the journal by scanning the + // disk, we don't allow finalizing incorrectly. + journal.close(); + journal = new Journal(TEST_LOG_DIR, JID, mockErrorReporter); + + try { + journal.finalizeLogSegment(makeRI(4), 1, 6); + fail("did not fail to finalize"); + } catch (JournalOutOfSyncException e) { + GenericTestUtils.assertExceptionContains( + "disk only contains up to txid 3", e); + } + } + + /** + * Ensure that finalizing a segment which doesn't exist throws the + * appropriate exception. + */ + @Test + public void testFinalizeMissingSegment() throws Exception { + journal.newEpoch(FAKE_NSINFO, 1); + try { + journal.finalizeLogSegment(makeRI(1), 1000, 1001); + fail("did not fail to finalize"); + } catch (JournalOutOfSyncException e) { + GenericTestUtils.assertExceptionContains( + "No log file to finalize at transaction ID 1000", e); + } + } + + /** + * Assume that a client is writing to a journal, but loses its connection + * in the middle of a segment. Thus, any future journal() calls in that + * segment may fail, because some txns were missed while the connection was + * down. + * + * Eventually, the connection comes back, and the NN tries to start a new + * segment at a higher txid. This should abort the old one and succeed. + */ + @Test + public void testAbortOldSegmentIfFinalizeIsMissed() throws Exception { + journal.newEpoch(FAKE_NSINFO, 1); + + // Start a segment at txid 1, and write a batch of 3 txns. + journal.startLogSegment(makeRI(1), 1); + journal.journal(makeRI(2), 1, 1, 3, + QJMTestUtil.createTxnData(1, 3)); + + GenericTestUtils.assertExists( + journal.getStorage().getInProgressEditLog(1)); + + // Try to start new segment at txid 6, this should abort old segment and + // then succeed, allowing us to write txid 6-9. + journal.startLogSegment(makeRI(3), 6); + journal.journal(makeRI(4), 6, 6, 3, + QJMTestUtil.createTxnData(6, 3)); + + // The old segment should *not* be finalized. + GenericTestUtils.assertExists( + journal.getStorage().getInProgressEditLog(1)); + GenericTestUtils.assertExists( + journal.getStorage().getInProgressEditLog(6)); + } + + /** + * Test behavior of startLogSegment() when a segment with the + * same transaction ID already exists. + */ + @Test + public void testStartLogSegmentWhenAlreadyExists() throws Exception { + journal.newEpoch(FAKE_NSINFO, 1); + + // Start a segment at txid 1, and write just 1 transaction. This + // would normally be the START_LOG_SEGMENT transaction. + journal.startLogSegment(makeRI(1), 1); + journal.journal(makeRI(2), 1, 1, 1, + QJMTestUtil.createTxnData(1, 1)); + + // Try to start new segment at txid 1, this should succeed, because + // we are allowed to re-start a segment if we only ever had the + // START_LOG_SEGMENT transaction logged. + journal.startLogSegment(makeRI(3), 1); + journal.journal(makeRI(4), 1, 1, 1, + QJMTestUtil.createTxnData(1, 1)); + + // This time through, write more transactions afterwards, simulating + // real user transactions. + journal.journal(makeRI(5), 1, 2, 3, + QJMTestUtil.createTxnData(2, 3)); + + try { + journal.startLogSegment(makeRI(6), 1); + fail("Did not fail to start log segment which would overwrite " + + "an existing one"); + } catch (IllegalStateException ise) { + GenericTestUtils.assertExceptionContains( + "seems to contain valid transactions", ise); + } + + journal.finalizeLogSegment(makeRI(7), 1, 4); + + // Ensure that we cannot overwrite a finalized segment + try { + journal.startLogSegment(makeRI(8), 1); + fail("Did not fail to start log segment which would overwrite " + + "an existing one"); + } catch (IllegalStateException ise) { + GenericTestUtils.assertExceptionContains( + "have a finalized segment", ise); + } + + } + + private static RequestInfo makeRI(int serial) { + return new RequestInfo(JID, 1, serial, 0); + } + + @Test + public void testNamespaceVerification() throws Exception { + journal.newEpoch(FAKE_NSINFO, 1); + + try { + journal.newEpoch(FAKE_NSINFO_2, 2); + fail("Did not fail newEpoch() when namespaces mismatched"); + } catch (IOException ioe) { + GenericTestUtils.assertExceptionContains( + "Incompatible namespaceID", ioe); + } + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java new file mode 100644 index 00000000000..f57f7e66b40 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java @@ -0,0 +1,332 @@ +/** + * 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.io.File; +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.fs.FileUtil; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.MiniDFSCluster; +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.MetricsRecordBuilder; +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.test.MetricsAsserts; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.base.Charsets; +import com.google.common.base.Stopwatch; +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); + + private JournalNode jn; + private Journal journal; + private Configuration conf = new Configuration(); + private IPCLoggerChannel ch; + private String journalId; + + static { + // Avoid an error when we double-initialize JvmMetrics + DefaultMetricsSystem.setMiniClusterMode(true); + } + + @Before + public void setup() throws Exception { + File editsDir = new File(MiniDFSCluster.getBaseDirectory() + + File.separator + "TestJournalNode"); + FileUtil.fullyDelete(editsDir); + + conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, + editsDir.getAbsolutePath()); + conf.set(DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_KEY, + "0.0.0.0:0"); + jn = new JournalNode(); + jn.setConf(conf); + jn.start(); + journalId = "test-journalid-" + GenericTestUtils.uniqueSequenceId(); + journal = jn.getOrCreateJournal(journalId); + journal.format(FAKE_NSINFO); + + ch = new IPCLoggerChannel(conf, FAKE_NSINFO, journalId, jn.getBoundIpcAddress()); + } + + @After + public void teardown() throws Exception { + jn.stop(0); + } + + @Test + public void testJournal() throws Exception { + MetricsRecordBuilder metrics = MetricsAsserts.getMetrics( + journal.getMetricsForTests().getName()); + MetricsAsserts.assertCounter("BatchesWritten", 0L, metrics); + MetricsAsserts.assertCounter("BatchesWrittenWhileLagging", 0L, metrics); + MetricsAsserts.assertGauge("CurrentLagTxns", 0L, metrics); + + IPCLoggerChannel ch = new IPCLoggerChannel( + conf, FAKE_NSINFO, journalId, jn.getBoundIpcAddress()); + ch.newEpoch(1).get(); + ch.setEpoch(1); + ch.startLogSegment(1).get(); + ch.sendEdits(1L, 1, 1, "hello".getBytes(Charsets.UTF_8)).get(); + + metrics = MetricsAsserts.getMetrics( + journal.getMetricsForTests().getName()); + MetricsAsserts.assertCounter("BatchesWritten", 1L, metrics); + MetricsAsserts.assertCounter("BatchesWrittenWhileLagging", 0L, metrics); + MetricsAsserts.assertGauge("CurrentLagTxns", 0L, metrics); + + ch.setCommittedTxId(100L); + ch.sendEdits(1L, 2, 1, "goodbye".getBytes(Charsets.UTF_8)).get(); + + metrics = MetricsAsserts.getMetrics( + journal.getMetricsForTests().getName()); + MetricsAsserts.assertCounter("BatchesWritten", 2L, metrics); + MetricsAsserts.assertCounter("BatchesWrittenWhileLagging", 1L, metrics); + MetricsAsserts.assertGauge("CurrentLagTxns", 98L, metrics); + + } + + + @Test + public void testReturnsSegmentInfoAtEpochTransition() throws Exception { + ch.newEpoch(1).get(); + ch.setEpoch(1); + ch.startLogSegment(1).get(); + ch.sendEdits(1L, 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); + // Because the new segment is empty, it is equivalent to not having + // started writing it. Hence, we should return the prior segment txid. + assertEquals(1, 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, journalId, jn.getBoundIpcAddress()); + ch.newEpoch(1).get(); + ch.setEpoch(1); + ch.startLogSegment(1).get(); + ch.sendEdits(1L, 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=" + journalId)); + 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=" + journalId); + HttpURLConnection connection = (HttpURLConnection)badUrl.openConnection(); + try { + assertEquals(404, connection.getResponseCode()); + } finally { + connection.disconnect(); + } + } + + /** + * Test that the JournalNode performs correctly as a Paxos + * Acceptor process. + */ + @Test + public void testAcceptRecoveryBehavior() throws Exception { + // We need to run newEpoch() first, or else we have no way to distinguish + // different proposals for the same decision. + try { + ch.prepareRecovery(1L).get(); + fail("Did not throw IllegalState when trying to run paxos without an epoch"); + } catch (ExecutionException ise) { + GenericTestUtils.assertExceptionContains("bad epoch", ise); + } + + ch.newEpoch(1).get(); + ch.setEpoch(1); + + // prepare() with no previously accepted value and no logs present + PrepareRecoveryResponseProto prep = ch.prepareRecovery(1L).get(); + System.err.println("Prep: " + prep); + assertFalse(prep.hasAcceptedInEpoch()); + assertFalse(prep.hasSegmentState()); + + // Make a log segment, and prepare again -- this time should see the + // segment existing. + ch.startLogSegment(1L).get(); + ch.sendEdits(1L, 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 + 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); + } + } + + @Test + public void testFailToStartWithBadConfig() throws Exception { + Configuration conf = new Configuration(); + conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, "non-absolute-path"); + assertJNFailsToStart(conf, "should be an absolute path"); + + // Existing file which is not a directory + conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, "/dev/null"); + assertJNFailsToStart(conf, "is not a directory"); + + // Directory which cannot be created + conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, "/proc/does-not-exist"); + assertJNFailsToStart(conf, "Could not create"); + + } + + private static void assertJNFailsToStart(Configuration conf, + String errString) { + try { + JournalNode jn = new JournalNode(); + jn.setConf(conf); + jn.start(); + } catch (Exception e) { + GenericTestUtils.assertExceptionContains(errString, e); + } + } + + /** + * Simple test of how fast the code path is to write edits. + * This isn't a true unit test, but can be run manually to + * check performance. + * + * At the time of development, this test ran in ~4sec on an + * SSD-enabled laptop (1.8ms/batch). + */ + @Test(timeout=100000) + public void testPerformance() throws Exception { + doPerfTest(8192, 1024); // 8MB + } + + private void doPerfTest(int editsSize, int numEdits) throws Exception { + byte[] data = new byte[editsSize]; + ch.newEpoch(1).get(); + ch.setEpoch(1); + ch.startLogSegment(1).get(); + + Stopwatch sw = new Stopwatch().start(); + for (int i = 1; i < numEdits; i++) { + ch.sendEdits(1L, i, 1, data).get(); + } + long time = sw.elapsedMillis(); + + System.err.println("Wrote " + numEdits + " batches of " + editsSize + + " bytes in " + time + "ms"); + float avgRtt = (float)time/(float)numEdits; + long throughput = ((long)numEdits * editsSize * 1000L)/time; + System.err.println("Time per batch: " + avgRtt + "ms"); + System.err.println("Throughput: " + throughput + " bytes/sec"); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java index f32b70dc3d1..6a9a071b43c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java @@ -74,6 +74,7 @@ import org.apache.hadoop.util.Time; import org.apache.log4j.Level; import org.junit.Assert; import org.junit.Assume; +import org.junit.Before; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -87,14 +88,6 @@ public class TestBlockToken { public static final Log LOG = LogFactory.getLog(TestBlockToken.class); private static final String ADDRESS = "0.0.0.0"; - static final String SERVER_PRINCIPAL_KEY = "test.ipc.server.principal"; - private static Configuration conf; - static { - conf = new Configuration(); - conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos"); - UserGroupInformation.setConfiguration(conf); - } - static { ((Log4JLogger) Client.LOG).getLogger().setLevel(Level.ALL); ((Log4JLogger) Server.LOG).getLogger().setLevel(Level.ALL); @@ -111,6 +104,13 @@ public class TestBlockToken { ExtendedBlock block1 = new ExtendedBlock("0", 0L); ExtendedBlock block2 = new ExtendedBlock("10", 10L); ExtendedBlock block3 = new ExtendedBlock("-10", -108L); + + @Before + public void disableKerberos() { + Configuration conf = new Configuration(); + conf.set(HADOOP_SECURITY_AUTHENTICATION, "simple"); + UserGroupInformation.setConfiguration(conf); + } private static class GetLengthAnswer implements Answer { @@ -215,8 +215,9 @@ public class TestBlockToken { tokenGenerationAndVerification(masterHandler, slaveHandler); } - private Server createMockDatanode(BlockTokenSecretManager sm, - Token token) throws IOException, ServiceException { + private static Server createMockDatanode(BlockTokenSecretManager sm, + Token token, Configuration conf) + throws IOException, ServiceException { ClientDatanodeProtocolPB mockDN = mock(ClientDatanodeProtocolPB.class); BlockTokenIdentifier id = sm.createIdentifier(); @@ -238,12 +239,16 @@ public class TestBlockToken { @Test public void testBlockTokenRpc() throws Exception { + Configuration conf = new Configuration(); + conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos"); + UserGroupInformation.setConfiguration(conf); + BlockTokenSecretManager sm = new BlockTokenSecretManager( blockKeyUpdateInterval, blockTokenLifetime, 0, "fake-pool", null); Token token = sm.generateToken(block3, EnumSet.allOf(BlockTokenSecretManager.AccessMode.class)); - final Server server = createMockDatanode(sm, token); + final Server server = createMockDatanode(sm, token, conf); server.start(); @@ -272,13 +277,17 @@ public class TestBlockToken { */ @Test public void testBlockTokenRpcLeak() throws Exception { + Configuration conf = new Configuration(); + conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos"); + UserGroupInformation.setConfiguration(conf); + Assume.assumeTrue(FD_DIR.exists()); BlockTokenSecretManager sm = new BlockTokenSecretManager( blockKeyUpdateInterval, blockTokenLifetime, 0, "fake-pool", null); Token token = sm.generateToken(block3, EnumSet.allOf(BlockTokenSecretManager.AccessMode.class)); - final Server server = createMockDatanode(sm, token); + final Server server = createMockDatanode(sm, token, conf); server.start(); final InetSocketAddress addr = NetUtils.getConnectAddress(server); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java index ada2e2a4054..f310959d9a1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java @@ -17,11 +17,16 @@ */ package org.apache.hadoop.hdfs.server.namenode; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.spy; + import java.io.File; import java.io.IOException; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.hadoop.fs.UnresolvedLinkException; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -182,6 +187,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. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java index f11d2a85392..16dd09957e2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java @@ -1220,7 +1220,7 @@ public class TestEditLog { elfos.create(); elfos.writeRaw(garbage, 0, garbage.length); elfos.setReadyToFlush(); - elfos.flushAndSync(); + elfos.flushAndSync(true); elfos.close(); elfos = null; file = new File(TEST_LOG_NAME); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java index 22ab02d2a9d..24446d655d8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java @@ -55,7 +55,7 @@ public class TestEditLogFileOutputStream { static void flushAndCheckLength(EditLogFileOutputStream elos, long expectedLength) throws IOException { elos.setReadyToFlush(); - elos.flushAndSync(); + elos.flushAndSync(true); assertEquals(expectedLength, elos.getFile().length()); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java index 23fd3b51a71..a8dac5701e4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java @@ -74,7 +74,7 @@ public class TestNameNodeRecovery { elts.addTransactionsToLog(elfos, cache); elfos.setReadyToFlush(); - elfos.flushAndSync(); + elfos.flushAndSync(true); elfos.close(); elfos = null; file = new File(TEST_LOG_NAME); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java index 47182d2798a..b534c03aa09 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java @@ -166,13 +166,13 @@ public class TestInitializeSharedEdits { } @Test - public void testDontOverWriteExistingDir() { + public void testDontOverWriteExistingDir() throws IOException { assertFalse(NameNode.initializeSharedEdits(conf, false)); assertTrue(NameNode.initializeSharedEdits(conf, false)); } @Test - public void testInitializeSharedEditsConfiguresGenericConfKeys() { + public void testInitializeSharedEditsConfiguresGenericConfKeys() throws IOException { Configuration conf = new Configuration(); conf.set(DFSConfigKeys.DFS_NAMESERVICES, "ns1"); conf.set(DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestBestEffortLongFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestBestEffortLongFile.java new file mode 100644 index 00000000000..c57dc97031f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestBestEffortLongFile.java @@ -0,0 +1,86 @@ +/** + * 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.util; + +import java.io.File; +import java.io.IOException; +import java.util.Random; + +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.io.IOUtils; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class TestBestEffortLongFile { + + private static final File FILE = new File(MiniDFSCluster.getBaseDirectory() + + File.separatorChar + "TestBestEffortLongFile"); + + @Before + public void cleanup() { + if (FILE.exists()) { + assertTrue(FILE.delete()); + } + FILE.getParentFile().mkdirs(); + } + + @Test + public void testGetSet() throws IOException { + BestEffortLongFile f = new BestEffortLongFile(FILE, 12345L); + try { + // Before the file exists, should return default. + assertEquals(12345L, f.get()); + + // And first access should open it. + assertTrue(FILE.exists()); + + Random r = new Random(); + for (int i = 0; i < 100; i++) { + long newVal = r.nextLong(); + // Changing the value should be reflected in the next get() call. + f.set(newVal); + assertEquals(newVal, f.get()); + + // And should be reflected in a new instance (ie it actually got + // written to the file) + BestEffortLongFile f2 = new BestEffortLongFile(FILE, 999L); + try { + assertEquals(newVal, f2.get()); + } finally { + IOUtils.closeStream(f2); + } + } + } finally { + IOUtils.closeStream(f); + } + } + + @Test + public void testTruncatedFileReturnsDefault() throws IOException { + assertTrue(FILE.createNewFile()); + assertEquals(0, FILE.length()); + BestEffortLongFile f = new BestEffortLongFile(FILE, 12345L); + try { + assertEquals(12345L, f.get()); + } finally { + f.close(); + } + } +} diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml index f449b5d9b40..ae2ab37cad4 100644 --- a/hadoop-project/src/site/site.xml +++ b/hadoop-project/src/site/site.xml @@ -54,7 +54,8 @@ - + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HDFSHighAvailability.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HDFSHighAvailabilityWithNFS.apt.vm similarity index 95% rename from hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HDFSHighAvailability.apt.vm rename to hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HDFSHighAvailabilityWithNFS.apt.vm index 7e7cb66772f..efa3f931bbc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HDFSHighAvailability.apt.vm +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HDFSHighAvailabilityWithNFS.apt.vm @@ -25,12 +25,21 @@ HDFS High Availability * {Purpose} This guide provides an overview of the HDFS High Availability (HA) feature and - how to configure and manage an HA HDFS cluster. + how to configure and manage an HA HDFS cluster, using NFS for the shared + storage required by the NameNodes. This document assumes that the reader has a general understanding of general components and node types in an HDFS cluster. Please refer to the HDFS Architecture guide for details. +* {Note: Using the Quorum Journal Manager or Conventional Shared Storage} + + This guide discusses how to configure and use HDFS HA using a shared NFS + directory to share edit logs between the Active and Standby NameNodes. For + information on how to configure HDFS HA using the Quorum Journal Manager + instead of NFS, please see {{{./HDFSHighAvailabilityWithQJM.html}this + alternative guide.}} + * {Background} Prior to Hadoop 2.0.0, the NameNode was a single point of failure (SPOF) in @@ -297,7 +306,7 @@ HDFS High Availability dfs.ha.fencing.ssh.connect-timeout - + 30000 --- @@ -375,17 +384,22 @@ HDFS High Availability ** Deployment details After all of the necessary configuration options have been set, one must - initially synchronize the two HA NameNodes' on-disk metadata. If you are - setting up a fresh HDFS cluster, you should first run the format command () on one of NameNodes. If you have already formatted the - NameNode, or are converting a non-HA-enabled cluster to be HA-enabled, you - should now copy over the contents of your NameNode metadata directories to - the other, unformatted NameNode using or a similar utility. The location - of the directories containing the NameNode metadata are configured via the - configuration options <> and/or - <>. At this time, you should also ensure that the - shared edits dir (as configured by <>) includes - all recent edits files which are in your NameNode metadata directories. + initially synchronize the two HA NameNodes' on-disk metadata. + + * If you are setting up a fresh HDFS cluster, you should first run the format + command () on one of NameNodes. + + * If you have already formatted the NameNode, or are converting a + non-HA-enabled cluster to be HA-enabled, you should now copy over the + contents of your NameNode metadata directories to the other, unformatted + NameNode by running the command "" on the + unformatted NameNode. Running this command will also ensure that the shared + edits directory (as configured by <>) contains + sufficient edits transactions to be able to start both NameNodes. + + * If you are converting a non-HA NameNode to be HA, you should run the + command "", which will initialize the shared + edits directory with the edits data from the local NameNode edits directories. At this point you may start both of your HA NameNodes as you normally would start a NameNode. @@ -863,4 +877,4 @@ $ zkCli.sh create /ledgers/available 0 3) Auto-Recovery of storage node failures. Work inprogress {{{https://issues.apache.org/jira/browse/BOOKKEEPER-237 }BOOKKEEPER-237}}. - Currently we have the tools to manually recover the data from failed storage nodes. \ No newline at end of file + Currently we have the tools to manually recover the data from failed storage nodes. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HDFSHighAvailabilityWithQJM.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HDFSHighAvailabilityWithQJM.apt.vm new file mode 100644 index 00000000000..2aefc3584c0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HDFSHighAvailabilityWithQJM.apt.vm @@ -0,0 +1,767 @@ +~~ 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. + + --- + Hadoop Distributed File System-${project.version} - High Availability + --- + --- + ${maven.build.timestamp} + +HDFS High Availability Using the Quorum Journal Manager + + \[ {{{./index.html}Go Back}} \] + +%{toc|section=1|fromDepth=0} + +* {Purpose} + + This guide provides an overview of the HDFS High Availability (HA) feature + and how to configure and manage an HA HDFS cluster, using the Quorum Journal + Manager (QJM) feature. + + This document assumes that the reader has a general understanding of + general components and node types in an HDFS cluster. Please refer to the + HDFS Architecture guide for details. + +* {Note: Using the Quorum Journal Manager or Conventional Shared Storage} + + This guide discusses how to configure and use HDFS HA using the Quorum + Journal Manager (QJM) to share edit logs between the Active and Standby + NameNodes. For information on how to configure HDFS HA using NFS for shared + storage instead of the QJM, please see + {{{./HDFSHighAvailabilityWithNFS.html}this alternative guide.}} + +* {Background} + + Prior to Hadoop 2.0.0, the NameNode was a single point of failure (SPOF) in + an HDFS cluster. Each cluster had a single NameNode, and if that machine or + process became unavailable, the cluster as a whole would be unavailable + until the NameNode was either restarted or brought up on a separate machine. + + This impacted the total availability of the HDFS cluster in two major ways: + + * In the case of an unplanned event such as a machine crash, the cluster would + be unavailable until an operator restarted the NameNode. + + * Planned maintenance events such as software or hardware upgrades on the + NameNode machine would result in windows of cluster downtime. + + The HDFS High Availability feature addresses the above problems by providing + the option of running two redundant NameNodes in the same cluster in an + Active/Passive configuration with a hot standby. This allows a fast failover to + a new NameNode in the case that a machine crashes, or a graceful + administrator-initiated failover for the purpose of planned maintenance. + +* {Architecture} + + In a typical HA cluster, two separate machines are configured as NameNodes. + At any point in time, exactly one of the NameNodes is in an state, + and the other is in a state. The Active NameNode is responsible + for all client operations in the cluster, while the Standby is simply acting + as a slave, maintaining enough state to provide a fast failover if + necessary. + + In order for the Standby node to keep its state synchronized with the Active + node, both nodes communicate with a group of separate daemons called + "JournalNodes" (JNs). When any namespace modification is performed by the + Active node, it durably logs a record of the modification to a majority of + these JNs. The Standby node is capable of reading the edits from the JNs, and + is constantly watching them for changes to the edit log. As the Standby Node + sees the edits, it applies them to its own namespace. In the event of a + failover, the Standby will ensure that it has read all of the edits from the + JounalNodes before promoting itself to the Active state. This ensures that the + namespace state is fully synchronized before a failover occurs. + + In order to provide a fast failover, it is also necessary that the Standby node + have up-to-date information regarding the location of blocks in the cluster. + In order to achieve this, the DataNodes are configured with the location of + both NameNodes, and send block location information and heartbeats to both. + + It is vital for the correct operation of an HA cluster that only one of the + NameNodes be Active at a time. Otherwise, the namespace state would quickly + diverge between the two, risking data loss or other incorrect results. In + order to ensure this property and prevent the so-called "split-brain scenario," + the JournalNodes will only ever allow a single NameNode to be a writer at a + time. During a failover, the NameNode which is to become active will simply + take over the role of writing to the JournalNodes, which will effectively + prevent the other NameNode from continuing in the Active state, allowing the + new Active to safely proceed with failover. + +* {Hardware resources} + + In order to deploy an HA cluster, you should prepare the following: + + * <> - the machines on which you run the Active and + Standby NameNodes should have equivalent hardware to each other, and + equivalent hardware to what would be used in a non-HA cluster. + + * <> - the machines on which you run the JournalNodes. + The JournalNode daemon is relatively lightweight, so these daemons may + reasonably be collocated on machines with other Hadoop daemons, for example + NameNodes, the JobTracker, or the YARN ResourceManager. <> There + must be at least 3 JournalNode daemons, since edit log modifications must be + written to a majority of JNs. This will allow the system to tolerate the + failure of a single machine. You may also run more than 3 JournalNodes, but + in order to actually increase the number of failures the system can tolerate, + you should run an odd number of JNs, (i.e. 3, 5, 7, etc.). Note that when + running with N JournalNodes, the system can tolerate at most (N - 1) / 2 + failures and continue to function normally. + + Note that, in an HA cluster, the Standby NameNode also performs checkpoints of + the namespace state, and thus it is not necessary to run a Secondary NameNode, + CheckpointNode, or BackupNode in an HA cluster. In fact, to do so would be an + error. This also allows one who is reconfiguring a non-HA-enabled HDFS cluster + to be HA-enabled to reuse the hardware which they had previously dedicated to + the Secondary NameNode. + +* {Deployment} + +** Configuration overview + + Similar to Federation configuration, HA configuration is backward compatible + and allows existing single NameNode configurations to work without change. + The new configuration is designed such that all the nodes in the cluster may + have the same configuration without the need for deploying different + configuration files to different machines based on the type of the node. + + Like HDFS Federation, HA clusters reuse the <<>> to identify a + single HDFS instance that may in fact consist of multiple HA NameNodes. In + addition, a new abstraction called <<>> is added with HA. Each + distinct NameNode in the cluster has a different NameNode ID to distinguish it. + To support a single configuration file for all of the NameNodes, the relevant + configuration parameters are suffixed with the <> as well as + the <>. + +** Configuration details + + To configure HA NameNodes, you must add several configuration options to your + <> configuration file. + + The order in which you set these configurations is unimportant, but the values + you choose for <> and + <> will determine the keys of those that + follow. Thus, you should decide on these values before setting the rest of the + configuration options. + + * <> - the logical name for this new nameservice + + Choose a logical name for this nameservice, for example "mycluster", and use + this logical name for the value of this config option. The name you choose is + arbitrary. It will be used both for configuration and as the authority + component of absolute HDFS paths in the cluster. + + <> If you are also using HDFS Federation, this configuration setting + should also include the list of other nameservices, HA or otherwise, as a + comma-separated list. + +---- + + dfs.nameservices + mycluster + +---- + + * <> - unique identifiers for each NameNode in the nameservice + + Configure with a list of comma-separated NameNode IDs. This will be used by + DataNodes to determine all the NameNodes in the cluster. For example, if you + used "mycluster" as the nameservice ID previously, and you wanted to use "nn1" + and "nn2" as the individual IDs of the NameNodes, you would configure this as + such: + +---- + + dfs.ha.namenodes.mycluster + nn1,nn2 + +---- + + <> Currently, only a maximum of two NameNodes may be configured per + nameservice. + + * <> - the fully-qualified RPC address for each NameNode to listen on + + For both of the previously-configured NameNode IDs, set the full address and + IPC port of the NameNode processs. Note that this results in two separate + configuration options. For example: + +---- + + dfs.namenode.rpc-address.mycluster.nn1 + machine1.example.com:8020 + + + dfs.namenode.rpc-address.mycluster.nn2 + machine2.example.com:8020 + +---- + + <> You may similarly configure the "<>" setting if + you so desire. + + * <> - the fully-qualified HTTP address for each NameNode to listen on + + Similarly to above, set the addresses for both NameNodes' HTTP + servers to listen on. For example: + +---- + + dfs.namenode.http-address.mycluster.nn1 + machine1.example.com:50070 + + + dfs.namenode.http-address.mycluster.nn2 + machine2.example.com:50070 + +---- + + <> If you have Hadoop's security features enabled, you should also set + the similarly for each NameNode. + + * <> - the URI which identifies the group of JNs where the NameNodes will write/read edits + + This is where one configures the addresses of the JournalNodes which provide + the shared edits storage, written to by the Active nameNode and read by the + Standby NameNode to stay up-to-date with all the file system changes the Active + NameNode makes. Though you must specify several JournalNode addresses, + <> The URI should be of the form: + "qjournal://;;/". The Journal + ID is a unique identifier for this nameservice, which allows a single set of + JournalNodes to provide storage for multiple federated namesystems. Though not + a requirement, it's a good idea to reuse the nameservice ID for the journal + identifier. + + For example, if the JournalNodes for this cluster were running on the + machines "node1.example.com", "node2.example.com", and "node3.example.com" and + the nameservice ID were "mycluster", you would use the following as the value + for this setting (the default port for the JournalNode is 8485): + +---- + + dfs.namenode.shared.edits.dir + qjournal://node1.example.com:8485;node2.example.com:8485;node3.example.com:8485/mycluster + +---- + + * <> - the Java class that HDFS clients use to contact the Active NameNode + + Configure the name of the Java class which will be used by the DFS Client to + determine which NameNode is the current Active, and therefore which NameNode is + currently serving client requests. The only implementation which currently + ships with Hadoop is the <>, so use this + unless you are using a custom one. For example: + +---- + + dfs.client.failover.proxy.provider.mycluster + org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider + +---- + + * <> - a list of scripts or Java classes which will be used to fence the Active NameNode during a failover + + It is desirable for correctness of the system that only one NameNode be in + the Active state at any given time. <> However, when a failover occurs, it is still + possible that the previous Active NameNode could serve read requests to + clients, which may be out of date until that NameNode shuts down when trying to + write to the JournalNodes. For this reason, it is still desirable to configure + some fencing methods even when using the Quorum Journal Manager. However, to + improve the availability of the system in the event the fencing mechanisms + fail, it is advisable to configure a fencing method which is guaranteed to + return success as the last fencing method in the list. Note that if you choose + to use no actual fencing methods, you still must configure something for this + setting, for example "<<>>". + + The fencing methods used during a failover are configured as a + carriage-return-separated list, which will be attempted in order until one + indicates that fencing has succeeded. There are two methods which ship with + Hadoop: and . For information on implementing your own custom + fencing method, see the class. + + * <> - SSH to the Active NameNode and kill the process + + The option SSHes to the target node and uses to kill the + process listening on the service's TCP port. In order for this fencing option + to work, it must be able to SSH to the target node without providing a + passphrase. Thus, one must also configure the + <> option, which is a + comma-separated list of SSH private key files. For example: + +--- + + dfs.ha.fencing.methods + sshfence + + + + dfs.ha.fencing.ssh.private-key-files + /home/exampleuser/.ssh/id_rsa + +--- + + Optionally, one may configure a non-standard username or port to perform the + SSH. One may also configure a timeout, in milliseconds, for the SSH, after + which this fencing method will be considered to have failed. It may be + configured like so: + +--- + + dfs.ha.fencing.methods + sshfence([[username][:port]]) + + + dfs.ha.fencing.ssh.connect-timeout + 30000 + +--- + + * <> - run an arbitrary shell command to fence the Active NameNode + + The fencing method runs an arbitrary shell command. It may be + configured like so: + +--- + + dfs.ha.fencing.methods + shell(/path/to/my/script.sh arg1 arg2 ...) + +--- + + The string between '(' and ')' is passed directly to a bash shell and may not + include any closing parentheses. + + The shell command will be run with an environment set up to contain all of the + current Hadoop configuration variables, with the '_' character replacing any + '.' characters in the configuration keys. The configuration used has already had + any namenode-specific configurations promoted to their generic forms -- for example + <> will contain the RPC address of the target node, even + though the configuration may specify that variable as + <>. + + Additionally, the following variables referring to the target node to be fenced + are also available: + +*-----------------------:-----------------------------------+ +| $target_host | hostname of the node to be fenced | +*-----------------------:-----------------------------------+ +| $target_port | IPC port of the node to be fenced | +*-----------------------:-----------------------------------+ +| $target_address | the above two, combined as host:port | +*-----------------------:-----------------------------------+ +| $target_nameserviceid | the nameservice ID of the NN to be fenced | +*-----------------------:-----------------------------------+ +| $target_namenodeid | the namenode ID of the NN to be fenced | +*-----------------------:-----------------------------------+ + + These environment variables may also be used as substitutions in the shell + command itself. For example: + +--- + + dfs.ha.fencing.methods + shell(/path/to/my/script.sh --nameservice=$target_nameserviceid $target_host:$target_port) + +--- + + If the shell command returns an exit + code of 0, the fencing is determined to be successful. If it returns any other + exit code, the fencing was not successful and the next fencing method in the + list will be attempted. + + <> This fencing method does not implement any timeout. If timeouts are + necessary, they should be implemented in the shell script itself (eg by forking + a subshell to kill its parent in some number of seconds). + + * <> - the default path prefix used by the Hadoop FS client when none is given + + Optionally, you may now configure the default path for Hadoop clients to use + the new HA-enabled logical URI. If you used "mycluster" as the nameservice ID + earlier, this will be the value of the authority portion of all of your HDFS + paths. This may be configured like so, in your <> file: + +--- + + fs.defaultFS + hdfs://mycluster + +--- + + + * <> - the path where the JournalNode daemon will store its local state + + This is the absolute path on the JournalNode machines where the edits and + other local state used by the JNs will be stored. You may only use a single + path for this configuration. Redundancy for this data is provided by running + multiple separate JournalNodes, or by configuring this directory on a + locally-attached RAID array. For example: + +--- + + dfs.journalnode.edits.dir + /path/to/journal/node/local/data + +--- + +** Deployment details + + After all of the necessary configuration options have been set, you must + start the JournalNode daemons on the set of machines where they will run. This + can be done by running the command "" and waiting + for the daemon to start on each of the relevant machines. + + Once the JournalNodes have been started, one must initially synchronize the + two HA NameNodes' on-disk metadata. + + * If you are setting up a fresh HDFS cluster, you should first run the format + command () on one of NameNodes. + + * If you have already formatted the NameNode, or are converting a + non-HA-enabled cluster to be HA-enabled, you should now copy over the + contents of your NameNode metadata directories to the other, unformatted + NameNode by running the command "" on the + unformatted NameNode. Running this command will also ensure that the + JournalNodes (as configured by <>) contain + sufficient edits transactions to be able to start both NameNodes. + + * If you are converting a non-HA NameNode to be HA, you should run the + command "", which will initialize the + JournalNodes with the edits data from the local NameNode edits directories. + + At this point you may start both of your HA NameNodes as you normally would + start a NameNode. + + You can visit each of the NameNodes' web pages separately by browsing to their + configured HTTP addresses. You should notice that next to the configured + address will be the HA state of the NameNode (either "standby" or "active".) + Whenever an HA NameNode starts, it is initially in the Standby state. + +** Administrative commands + + Now that your HA NameNodes are configured and started, you will have access + to some additional commands to administer your HA HDFS cluster. Specifically, + you should familiarize yourself with all of the subcommands of the "" command. Running this command without any additional arguments will + display the following usage information: + +--- +Usage: DFSHAAdmin [-ns ] + [-transitionToActive ] + [-transitionToStandby ] + [-failover [--forcefence] [--forceactive] ] + [-getServiceState ] + [-checkHealth ] + [-help ] +--- + + This guide describes high-level uses of each of these subcommands. For + specific usage information of each subcommand, you should run ">". + + * <> and <> - transition the state of the given NameNode to Active or Standby + + These subcommands cause a given NameNode to transition to the Active or Standby + state, respectively. <> Instead, one should almost always prefer to + use the "" subcommand. + + * <> - initiate a failover between two NameNodes + + This subcommand causes a failover from the first provided NameNode to the + second. If the first NameNode is in the Standby state, this command simply + transitions the second to the Active state without error. If the first NameNode + is in the Active state, an attempt will be made to gracefully transition it to + the Standby state. If this fails, the fencing methods (as configured by + <>) will be attempted in order until one + succeeds. Only after this process will the second NameNode be transitioned to + the Active state. If no fencing method succeeds, the second NameNode will not + be transitioned to the Active state, and an error will be returned. + + * <> - determine whether the given NameNode is Active or Standby + + Connect to the provided NameNode to determine its current state, printing + either "standby" or "active" to STDOUT appropriately. This subcommand might be + used by cron jobs or monitoring scripts which need to behave differently based + on whether the NameNode is currently Active or Standby. + + * <> - check the health of the given NameNode + + Connect to the provided NameNode to check its health. The NameNode is capable + of performing some diagnostics on itself, including checking if internal + services are running as expected. This command will return 0 if the NameNode is + healthy, non-zero otherwise. One might use this command for monitoring + purposes. + + <> This is not yet implemented, and at present will always return + success, unless the given NameNode is completely down. + +* {Automatic Failover} + +** Introduction + + The above sections describe how to configure manual failover. In that mode, + the system will not automatically trigger a failover from the active to the + standby NameNode, even if the active node has failed. This section describes + how to configure and deploy automatic failover. + +** Components + + Automatic failover adds two new components to an HDFS deployment: a ZooKeeper + quorum, and the ZKFailoverController process (abbreviated as ZKFC). + + Apache ZooKeeper is a highly available service for maintaining small amounts + of coordination data, notifying clients of changes in that data, and + monitoring clients for failures. The implementation of automatic HDFS failover + relies on ZooKeeper for the following things: + + * <> - each of the NameNode machines in the cluster + maintains a persistent session in ZooKeeper. If the machine crashes, the + ZooKeeper session will expire, notifying the other NameNode that a failover + should be triggered. + + * <> - ZooKeeper provides a simple mechanism to + exclusively elect a node as active. If the current active NameNode crashes, + another node may take a special exclusive lock in ZooKeeper indicating that + it should become the next active. + + The ZKFailoverController (ZKFC) is a new component which is a ZooKeeper client + which also monitors and manages the state of the NameNode. Each of the + machines which runs a NameNode also runs a ZKFC, and that ZKFC is responsible + for: + + * <> - the ZKFC pings its local NameNode on a periodic + basis with a health-check command. So long as the NameNode responds in a + timely fashion with a healthy status, the ZKFC considers the node + healthy. If the node has crashed, frozen, or otherwise entered an unhealthy + state, the health monitor will mark it as unhealthy. + + * <> - when the local NameNode is healthy, the + ZKFC holds a session open in ZooKeeper. If the local NameNode is active, it + also holds a special "lock" znode. This lock uses ZooKeeper's support for + "ephemeral" nodes; if the session expires, the lock node will be + automatically deleted. + + * <> - if the local NameNode is healthy, and the + ZKFC sees that no other node currently holds the lock znode, it will itself + try to acquire the lock. If it succeeds, then it has "won the election", and + is responsible for running a failover to make its local NameNode active. The + failover process is similar to the manual failover described above: first, + the previous active is fenced if necessary, and then the local NameNode + transitions to active state. + + For more details on the design of automatic failover, refer to the design + document attached to HDFS-2185 on the Apache HDFS JIRA. + +** Deploying ZooKeeper + + In a typical deployment, ZooKeeper daemons are configured to run on three or + five nodes. Since ZooKeeper itself has light resource requirements, it is + acceptable to collocate the ZooKeeper nodes on the same hardware as the HDFS + NameNode and Standby Node. Many operators choose to deploy the third ZooKeeper + process on the same node as the YARN ResourceManager. It is advisable to + configure the ZooKeeper nodes to store their data on separate disk drives from + the HDFS metadata for best performance and isolation. + + The setup of ZooKeeper is out of scope for this document. We will assume that + you have set up a ZooKeeper cluster running on three or more nodes, and have + verified its correct operation by connecting using the ZK CLI. + +** Before you begin + + Before you begin configuring automatic failover, you should shut down your + cluster. It is not currently possible to transition from a manual failover + setup to an automatic failover setup while the cluster is running. + +** Configuring automatic failover + + The configuration of automatic failover requires the addition of two new + parameters to your configuration. In your <<>> file, add: + +---- + + dfs.ha.automatic-failover.enabled + true + +---- + + This specifies that the cluster should be set up for automatic failover. + In your <<>> file, add: + +---- + + ha.zookeeper.quorum + zk1.example.com:2181,zk2.example.com:2181,zk3.example.com:2181 + +---- + + This lists the host-port pairs running the ZooKeeper service. + + As with the parameters described earlier in the document, these settings may + be configured on a per-nameservice basis by suffixing the configuration key + with the nameservice ID. For example, in a cluster with federation enabled, + you can explicitly enable automatic failover for only one of the nameservices + by setting <<>>. + + There are also several other configuration parameters which may be set to + control the behavior of automatic failover; however, they are not necessary + for most installations. Please refer to the configuration key specific + documentation for details. + +** Initializing HA state in ZooKeeper + + After the configuration keys have been added, the next step is to initialize + required state in ZooKeeper. You can do so by running the following command + from one of the NameNode hosts. + +---- +$ hdfs zkfc -formatZK +---- + + This will create a znode in ZooKeeper inside of which the automatic failover + system stores its data. + +** Starting the cluster with <<>> + + Since automatic failover has been enabled in the configuration, the + <<>> script will now automatically start a ZKFC daemon on any + machine that runs a NameNode. When the ZKFCs start, they will automatically + select one of the NameNodes to become active. + +** Starting the cluster manually + + If you manually manage the services on your cluster, you will need to manually + start the <<>> daemon on each of the machines that runs a NameNode. You + can start the daemon by running: + +---- +$ hadoop-daemon.sh start zkfc +---- + +** Securing access to ZooKeeper + + If you are running a secure cluster, you will likely want to ensure that the + information stored in ZooKeeper is also secured. This prevents malicious + clients from modifying the metadata in ZooKeeper or potentially triggering a + false failover. + + In order to secure the information in ZooKeeper, first add the following to + your <<>> file: + +---- + + ha.zookeeper.auth + @/path/to/zk-auth.txt + + + ha.zookeeper.acl + @/path/to/zk-acl.txt + +---- + + Please note the '@' character in these values -- this specifies that the + configurations are not inline, but rather point to a file on disk. + + The first configured file specifies a list of ZooKeeper authentications, in + the same format as used by the ZK CLI. For example, you may specify something + like: + +---- +digest:hdfs-zkfcs:mypassword +---- + ...where <<>> is a unique username for ZooKeeper, and + <<>> is some unique string used as a password. + + Next, generate a ZooKeeper ACL that corresponds to this authentication, using + a command like the following: + +---- +$ java -cp $ZK_HOME/lib/*:$ZK_HOME/zookeeper-3.4.2.jar org.apache.zookeeper.server.auth.DigestAuthenticationProvider hdfs-zkfcs:mypassword +output: hdfs-zkfcs:mypassword->hdfs-zkfcs:P/OQvnYyU/nF/mGYvB/xurX8dYs= +---- + + Copy and paste the section of this output after the '->' string into the file + <<>>, prefixed by the string "<<>>". For example: + +---- +digest:hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM=:rwcda +---- + + In order for these ACLs to take effect, you should then rerun the + <<>> command as described above. + + After doing so, you may verify the ACLs from the ZK CLI as follows: + +---- +[zk: localhost:2181(CONNECTED) 1] getAcl /hadoop-ha +'digest,'hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM= +: cdrwa +---- + +** Verifying automatic failover + + Once automatic failover has been set up, you should test its operation. To do + so, first locate the active NameNode. You can tell which node is active by + visiting the NameNode web interfaces -- each node reports its HA state at the + top of the page. + + Once you have located your active NameNode, you may cause a failure on that + node. For example, you can use <<>>> to simulate a JVM + crash. Or, you could power cycle the machine or unplug its network interface + to simulate a different kind of outage. After triggering the outage you wish + to test, the other NameNode should automatically become active within several + seconds. The amount of time required to detect a failure and trigger a + fail-over depends on the configuration of + <<>>, but defaults to 5 seconds. + + If the test does not succeed, you may have a misconfiguration. Check the logs + for the <<>> daemons as well as the NameNode daemons in order to further + diagnose the issue. + + +* Automatic Failover FAQ + + * <> + + No. On any given node you may start the ZKFC before or after its corresponding + NameNode. + + * <> + + You should add monitoring on each host that runs a NameNode to ensure that the + ZKFC remains running. In some types of ZooKeeper failures, for example, the + ZKFC may unexpectedly exit, and should be restarted to ensure that the system + is ready for automatic failover. + + Additionally, you should monitor each of the servers in the ZooKeeper + quorum. If ZooKeeper crashes, then automatic failover will not function. + + * <> + + If the ZooKeeper cluster crashes, no automatic failovers will be triggered. + However, HDFS will continue to run without any impact. When ZooKeeper is + restarted, HDFS will reconnect with no issues. + + * <> + + No. Currently, this is not supported. Whichever NameNode is started first will + become active. You may choose to start the cluster in a specific order such + that your preferred node starts first. + + * <> + + Even if automatic failover is configured, you may initiate a manual failover + using the same <<>> command. It will perform a coordinated + failover.