Merge HDFS-3077 into trunk

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1396943 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Todd Lipcon 2012-10-11 08:11:51 +00:00
commit adb50a3537
88 changed files with 10629 additions and 142 deletions

View File

@ -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

View File

@ -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.</description>
</property>
<property>
<name>security.qjournal.service.protocol.acl</name>
<value>${HADOOP_HDFS_USER}</value>
<description>ACL for QJournalProtocol, used by the NN to communicate with
JNs when using the QuorumJournalManager for edit logs.</description>
</property>
</configuration>

View File

@ -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.

View File

@ -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)

View File

@ -8,6 +8,9 @@
<Match>
<Package name="org.apache.hadoop.hdfs.server.namenode.ha.proto" />
</Match>
<Match>
<Package name="org.apache.hadoop.hdfs.qjournal.protocol" />
</Match>
<Match>
<Bug pattern="EI_EXPOSE_REP" />
</Match>
@ -273,10 +276,18 @@
<Method name="quit" />
<Bug pattern="DM_EXIT" />
</Match>
<!-- Don't complain about recoverBlock equality check -->
<Match>
<Class name="org.apache.hadoop.hdfs.server.datanode.DataNode" />
<Method name="recoverBlock" />
<Bug pattern="EC_UNRELATED_TYPES" />
</Match>
<!-- More complex cleanup logic confuses findbugs -->
<Match>
<Class name="org.apache.hadoop.hdfs.qjournal.server.Journal" />
<Method name="persistPaxosData" />
<Bug pattern="OS_OPEN_STREAM" />
</Match>
</FindBugsFilter>

View File

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

View File

@ -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();

View File

@ -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"

View File

@ -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;
}

View File

@ -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<String> getAllNnPrincipals(Configuration conf) throws IOException {
Set<String> principals = new HashSet<String>();
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.

View File

@ -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,

View File

@ -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(

View File

@ -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:
*
* <ul>
* <li>All methods return {@link ListenableFuture}s instead of synchronous
* objects.</li>
* <li>The {@link RequestInfo} objects are created by the underlying
* implementation.</li>
* </ul>
*/
interface AsyncLogger {
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<Void> 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<Void> startLogSegment(long txid);
/**
* Finalize a log segment.
*
* @param startTxId the first txid that was written to the segment
* @param endTxId the last txid that was written to the segment
*/
public ListenableFuture<Void> finalizeLogSegment(
long startTxId, long endTxId);
/**
* Allow the remote node to purge edit logs earlier than this.
* @param minTxIdToKeep the min txid which must be retained
*/
public ListenableFuture<Void> purgeLogsOlderThan(long minTxIdToKeep);
/**
* Format the log directory.
* @param nsInfo the namespace info to format with
*/
public ListenableFuture<Void> format(NamespaceInfo nsInfo);
/**
* @return whether or not the remote node has any valid data.
*/
public ListenableFuture<Boolean> isFormatted();
/**
* @return the state of the last epoch on the target node.
*/
public ListenableFuture<GetJournalStateResponseProto> getJournalState();
/**
* Begin a new epoch on the target node.
*/
public ListenableFuture<NewEpochResponseProto> newEpoch(long epoch);
/**
* Fetch the list of edit logs available on the remote node.
*/
public ListenableFuture<RemoteEditLogManifest> getEditLogManifest(
long fromTxnId);
/**
* Prepare recovery. See the HDFS-3077 design document for details.
*/
public ListenableFuture<PrepareRecoveryResponseProto> prepareRecovery(
long segmentTxId);
/**
* Accept a recovery proposal. See the HDFS-3077 design document for details.
*/
public ListenableFuture<Void> acceptRecovery(SegmentStateProto log,
URL fromUrl);
/**
* Set the epoch number used for all future calls.
*/
public void setEpoch(long e);
/**
* 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 <em>this</em>
* 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);
}

View File

@ -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<AsyncLogger> loggers;
private static final long INVALID_EPOCH = -1;
private long myEpoch = INVALID_EPOCH;
public AsyncLoggerSet(List<AsyncLogger> 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
*/
<V> Map<AsyncLogger, V> waitForWriteQuorum(QuorumCall<AsyncLogger, V> 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("<table class=\"storage\">");
sb.append("<thead><tr><td>JN</td><td>Status</td></tr></thead>\n");
for (AsyncLogger l : loggers) {
sb.append("<tr>");
sb.append("<td>" + JspUtil.escapeXml(l.toString()) + "</td>");
sb.append("<td>");
l.appendHtmlReport(sb);
sb.append("</td></tr>\n");
}
sb.append("</table>");
}
/**
* @return the (mutable) list of loggers, for use in tests to
* set up spies
*/
@VisibleForTesting
List<AsyncLogger> getLoggersForTests() {
return loggers;
}
///////////////////////////////////////////////////////////////////////////
// The rest of this file is simply boilerplate wrappers which fan-out the
// various IPC calls to the underlying AsyncLoggers and wrap the result
// in a QuorumCall.
///////////////////////////////////////////////////////////////////////////
public QuorumCall<AsyncLogger, GetJournalStateResponseProto> getJournalState() {
Map<AsyncLogger, ListenableFuture<GetJournalStateResponseProto>> calls =
Maps.newHashMap();
for (AsyncLogger logger : loggers) {
calls.put(logger, logger.getJournalState());
}
return QuorumCall.create(calls);
}
public QuorumCall<AsyncLogger, Boolean> isFormatted() {
Map<AsyncLogger, ListenableFuture<Boolean>> calls = Maps.newHashMap();
for (AsyncLogger logger : loggers) {
calls.put(logger, logger.isFormatted());
}
return QuorumCall.create(calls);
}
public QuorumCall<AsyncLogger,NewEpochResponseProto> newEpoch(
NamespaceInfo nsInfo,
long epoch) {
Map<AsyncLogger, ListenableFuture<NewEpochResponseProto>> calls =
Maps.newHashMap();
for (AsyncLogger logger : loggers) {
calls.put(logger, logger.newEpoch(epoch));
}
return QuorumCall.create(calls);
}
public QuorumCall<AsyncLogger, Void> startLogSegment(
long txid) {
Map<AsyncLogger, ListenableFuture<Void>> calls = Maps.newHashMap();
for (AsyncLogger logger : loggers) {
calls.put(logger, logger.startLogSegment(txid));
}
return QuorumCall.create(calls);
}
public QuorumCall<AsyncLogger, Void> finalizeLogSegment(long firstTxId,
long lastTxId) {
Map<AsyncLogger, ListenableFuture<Void>> calls = Maps.newHashMap();
for (AsyncLogger logger : loggers) {
calls.put(logger, logger.finalizeLogSegment(firstTxId, lastTxId));
}
return QuorumCall.create(calls);
}
public QuorumCall<AsyncLogger, Void> sendEdits(
long segmentTxId, long firstTxnId, int numTxns, byte[] data) {
Map<AsyncLogger, ListenableFuture<Void>> calls = Maps.newHashMap();
for (AsyncLogger logger : loggers) {
ListenableFuture<Void> future =
logger.sendEdits(segmentTxId, firstTxnId, numTxns, data);
calls.put(logger, future);
}
return QuorumCall.create(calls);
}
public QuorumCall<AsyncLogger, RemoteEditLogManifest>
getEditLogManifest(long fromTxnId) {
Map<AsyncLogger,
ListenableFuture<RemoteEditLogManifest>> calls
= Maps.newHashMap();
for (AsyncLogger logger : loggers) {
ListenableFuture<RemoteEditLogManifest> future =
logger.getEditLogManifest(fromTxnId);
calls.put(logger, future);
}
return QuorumCall.create(calls);
}
QuorumCall<AsyncLogger, PrepareRecoveryResponseProto>
prepareRecovery(long segmentTxId) {
Map<AsyncLogger,
ListenableFuture<PrepareRecoveryResponseProto>> calls
= Maps.newHashMap();
for (AsyncLogger logger : loggers) {
ListenableFuture<PrepareRecoveryResponseProto> future =
logger.prepareRecovery(segmentTxId);
calls.put(logger, future);
}
return QuorumCall.create(calls);
}
QuorumCall<AsyncLogger,Void>
acceptRecovery(SegmentStateProto log, URL fromURL) {
Map<AsyncLogger, ListenableFuture<Void>> calls
= Maps.newHashMap();
for (AsyncLogger logger : loggers) {
ListenableFuture<Void> future =
logger.acceptRecovery(log, fromURL);
calls.put(logger, future);
}
return QuorumCall.create(calls);
}
QuorumCall<AsyncLogger,Void> format(NamespaceInfo nsInfo) {
Map<AsyncLogger, ListenableFuture<Void>> calls =
Maps.newHashMap();
for (AsyncLogger logger : loggers) {
ListenableFuture<Void> future =
logger.format(nsInfo);
calls.put(logger, future);
}
return QuorumCall.create(calls);
}
}

View File

@ -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<QJournalProtocol>() {
@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<Boolean> isFormatted() {
return executor.submit(new Callable<Boolean>() {
@Override
public Boolean call() throws IOException {
return getProxy().isFormatted(journalId);
}
});
}
@Override
public ListenableFuture<GetJournalStateResponseProto> getJournalState() {
return executor.submit(new Callable<GetJournalStateResponseProto>() {
@Override
public GetJournalStateResponseProto call() throws IOException {
GetJournalStateResponseProto ret =
getProxy().getJournalState(journalId);
httpPort = ret.getHttpPort();
return ret;
}
});
}
@Override
public ListenableFuture<NewEpochResponseProto> newEpoch(
final long epoch) {
return executor.submit(new Callable<NewEpochResponseProto>() {
@Override
public NewEpochResponseProto call() throws IOException {
return getProxy().newEpoch(journalId, nsInfo, epoch);
}
});
}
@Override
public ListenableFuture<Void> sendEdits(
final long 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<Void> ret = null;
try {
ret = executor.submit(new Callable<Void>() {
@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<Void>() {
@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<Void> format(final NamespaceInfo nsInfo) {
return executor.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
getProxy().format(journalId, nsInfo);
return null;
}
});
}
@Override
public ListenableFuture<Void> startLogSegment(final long txid) {
return executor.submit(new Callable<Void>() {
@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<Void> finalizeLogSegment(
final long startTxId, final long endTxId) {
return executor.submit(new Callable<Void>() {
@Override
public Void call() throws IOException {
throwIfOutOfSync();
getProxy().finalizeLogSegment(createReqInfo(), startTxId, endTxId);
return null;
}
});
}
@Override
public ListenableFuture<Void> purgeLogsOlderThan(final long minTxIdToKeep) {
return executor.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
getProxy().purgeLogsOlderThan(createReqInfo(), minTxIdToKeep);
return null;
}
});
}
@Override
public ListenableFuture<RemoteEditLogManifest> getEditLogManifest(
final long fromTxnId) {
return executor.submit(new Callable<RemoteEditLogManifest>() {
@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<PrepareRecoveryResponseProto> prepareRecovery(
final long segmentTxId) {
return executor.submit(new Callable<PrepareRecoveryResponseProto>() {
@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<Void> acceptRecovery(
final SegmentStateProto log, final URL url) {
return executor.submit(new Callable<Void>() {
@Override
public Void call() throws IOException {
getProxy().acceptRecovery(createReqInfo(), log, url);
return null;
}
});
}
@Override
public String toString() {
return 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);
}
}

View File

@ -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<String, IPCLoggerChannelMetrics> 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);
}
}
}
}

View File

@ -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;
}

View File

@ -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 <KEY> a key used to identify each of the outgoing calls
* @param <RESULT> the type of the call result
*/
class QuorumCall<KEY, RESULT> {
private final Map<KEY, RESULT> successes = Maps.newHashMap();
private final Map<KEY, Throwable> exceptions = Maps.newHashMap();
/**
* 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 <KEY, RESULT> QuorumCall<KEY, RESULT> create(
Map<KEY, ? extends ListenableFuture<RESULT>> calls) {
final QuorumCall<KEY, RESULT> qr = new QuorumCall<KEY, RESULT>();
for (final Entry<KEY, ? extends ListenableFuture<RESULT>> e : calls.entrySet()) {
Preconditions.checkArgument(e.getValue() != null,
"null future for key: " + e.getKey());
Futures.addCallback(e.getValue(), new FutureCallback<RESULT>() {
@Override
public void onFailure(Throwable t) {
qr.addException(e.getKey(), t);
}
@Override
public void onSuccess(RESULT res) {
qr.addResult(e.getKey(), res);
}
});
}
return qr;
}
private QuorumCall() {
// Only instantiated from factory method above
}
/**
* Wait for the quorum to achieve a certain number of responses.
*
* Note that, even after this returns, more responses may arrive,
* causing the return value of other methods in this class to change.
*
* @param minResponses return as soon as this many responses have been
* received, regardless of whether they are successes or exceptions
* @param minSuccesses return as soon as this many successful (non-exception)
* responses have been received
* @param maxExceptions return as soon as this many exception responses
* have been received. Pass 0 to return immediately if any exception is
* received.
* @param millis the number of milliseconds to wait for
* @throws InterruptedException if the thread is interrupted while waiting
* @throws TimeoutException if the specified timeout elapses before
* achieving the desired conditions
*/
public synchronized void waitFor(
int minResponses, int minSuccesses, int maxExceptions,
int millis, 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<KEY, RESULT> getResults() {
return Maps.newHashMap(successes);
}
public synchronized void rethrowException(String msg) throws QuorumException {
Preconditions.checkState(!exceptions.isEmpty());
throw QuorumException.create(msg, successes, exceptions);
}
public static <K> String mapToString(
Map<K, ? extends Message> map) {
StringBuilder sb = new StringBuilder();
boolean first = true;
for (Map.Entry<K, ? extends Message> e : map.entrySet()) {
if (!first) {
sb.append("\n");
}
first = false;
sb.append(e.getKey()).append(": ")
.append(TextFormat.shortDebugString(e.getValue()));
}
return sb.toString();
}
}

View File

@ -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 <K> the keys for the quorum calls
* @param <V> the success response type
* @param successes any successful responses returned
* @param exceptions the exceptions returned
*/
public static <K, V> QuorumException create(
String simpleMsg,
Map<K, V> successes,
Map<K, Throwable> exceptions) {
Preconditions.checkArgument(!exceptions.isEmpty(),
"Must pass exceptions");
StringBuilder msg = new StringBuilder();
msg.append(simpleMsg).append(". ");
if (!successes.isEmpty()) {
msg.append(successes.size()).append(" successful responses:\n");
Joiner.on("\n")
.useForNull("null [success]")
.withKeyValueSeparator(": ")
.appendTo(msg, successes);
msg.append("\n");
}
msg.append(exceptions.size() + " exceptions thrown:\n");
boolean isFirst = true;
for (Map.Entry<K, Throwable> e : exceptions.entrySet()) {
if (!isFirst) {
msg.append("\n");
}
isFirst = false;
msg.append(e.getKey()).append(": ");
if (e.getValue() instanceof RuntimeException) {
msg.append(StringUtils.stringifyException(e.getValue()));
} else 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;
}

View File

@ -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<AsyncLogger> 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<AsyncLogger, NewEpochResponseProto> createNewUniqueEpoch()
throws IOException {
Preconditions.checkState(!loggers.isEpochEstablished(),
"epoch already created");
Map<AsyncLogger, GetJournalStateResponseProto> 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<AsyncLogger, NewEpochResponseProto> resps =
loggers.waitForWriteQuorum(loggers.newEpoch(nsInfo, myEpoch),
newEpochTimeoutMs, "newEpoch(" + myEpoch + ")");
loggers.setEpoch(myEpoch);
return resps;
}
@Override
public void format(NamespaceInfo nsInfo) throws IOException {
QuorumCall<AsyncLogger,Void> 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<AsyncLogger, Boolean> 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:
* <ul>
* <li>This segment will be finalized on a majority
* of nodes.</li>
* <li>All nodes which contain the finalized segment will
* agree on the length.</li>
* </ul>
*
* @param segmentTxId the starting txid of the segment
* @throws IOException
*/
private void recoverUnclosedSegment(long segmentTxId) throws IOException {
Preconditions.checkArgument(segmentTxId > 0);
LOG.info("Beginning recovery of unclosed segment starting at txid " +
segmentTxId);
// Step 1. Prepare recovery
QuorumCall<AsyncLogger,PrepareRecoveryResponseProto> prepare =
loggers.prepareRecovery(segmentTxId);
Map<AsyncLogger, PrepareRecoveryResponseProto> prepareResponses=
loggers.waitForWriteQuorum(prepare, prepareRecoveryTimeoutMs,
"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<AsyncLogger, PrepareRecoveryResponseProto> 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<AsyncLogger, PrepareRecoveryResponseProto> 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<AsyncLogger,Void> 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<AsyncLogger, Void> finalize =
loggers.finalizeLogSegment(logToSync.getStartTxId(), logToSync.getEndTxId());
loggers.waitForWriteQuorum(finalize, finalizeSegmentTimeoutMs,
String.format("finalizeLogSegment(%s-%s)",
logToSync.getStartTxId(),
logToSync.getEndTxId()));
}
static List<AsyncLogger> createLoggers(Configuration conf,
URI uri, NamespaceInfo nsInfo, AsyncLogger.Factory factory)
throws IOException {
List<AsyncLogger> ret = Lists.newArrayList();
List<InetSocketAddress> addrs = getLoggerAddresses(uri);
String jid = parseJournalId(uri);
for (InetSocketAddress addr : addrs) {
ret.add(factory.createLogger(conf, nsInfo, jid, addr));
}
return ret;
}
private static List<InetSocketAddress> getLoggerAddresses(URI uri)
throws IOException {
String authority = uri.getAuthority();
Preconditions.checkArgument(authority != null && !authority.isEmpty(),
"URI has no authority: " + uri);
String[] parts = StringUtils.split(authority, ';');
for (int i = 0; i < parts.length; i++) {
parts[i] = parts[i].trim();
}
if (parts.length % 2 == 0) {
LOG.warn("Quorum journal URI '" + uri + "' has an even number " +
"of Journal Nodes specified. This is not recommended!");
}
List<InetSocketAddress> addrs = Lists.newArrayList();
for (String addr : parts) {
addrs.add(NetUtils.createSocketAddr(
addr, DFSConfigKeys.DFS_JOURNALNODE_RPC_PORT_DEFAULT));
}
return addrs;
}
@Override
public EditLogOutputStream startLogSegment(long txId) throws IOException {
Preconditions.checkState(isActiveWriter,
"must recover segments before starting a new one");
QuorumCall<AsyncLogger,Void> q = loggers.startLogSegment(txId);
loggers.waitForWriteQuorum(q, startSegmentTimeoutMs,
"startLogSegment(" + txId + ")");
return new QuorumOutputStream(loggers, txId,
outputBufferCapacity, writeTxnsTimeoutMs);
}
@Override
public void finalizeLogSegment(long firstTxId, long lastTxId)
throws IOException {
QuorumCall<AsyncLogger,Void> 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<AsyncLogger, NewEpochResponseProto> 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<EditLogInputStream> streams,
long fromTxnId, boolean inProgressOk) throws IOException {
QuorumCall<AsyncLogger, RemoteEditLogManifest> q =
loggers.getEditLogManifest(fromTxnId);
Map<AsyncLogger, RemoteEditLogManifest> resps =
loggers.waitForWriteQuorum(q, selectInputStreamsTimeoutMs,
"selectInputStreams");
LOG.debug("selectInputStream manifests:\n" +
Joiner.on("\n").withKeyValueSeparator(": ").join(resps));
final PriorityQueue<EditLogInputStream> allStreams =
new PriorityQueue<EditLogInputStream>(64,
JournalSet.EDIT_LOG_INPUT_STREAM_COMPARATOR);
for (Map.Entry<AsyncLogger, RemoteEditLogManifest> e : resps.entrySet()) {
AsyncLogger logger = e.getKey();
RemoteEditLogManifest manifest = e.getValue();
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;
}
}

View File

@ -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<AsyncLogger, Void> 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 + "<br/>\n");
loggers.appendHtmlReport(sb);
return sb.toString();
}
@Override
public String toString() {
return "QuorumOutputStream starting at txid " + segmentTxId;
}
}

View File

@ -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<Entry<AsyncLogger, PrepareRecoveryResponseProto>> {
static final SegmentRecoveryComparator INSTANCE = new SegmentRecoveryComparator();
@Override
public int compare(
Entry<AsyncLogger, PrepareRecoveryResponseProto> a,
Entry<AsyncLogger, PrepareRecoveryResponseProto> 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();
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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;
}

View File

@ -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);
}
}

View File

@ -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 {
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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:
* <ul>
* <li>The QuorumJournalManager, when reading edits, fetches the edit streams
* from the journal nodes.</li>
* <li>During edits synchronization, one journal node will fetch edits from
* another journal node.</li>
* </ul>
*/
@InterfaceAudience.Private
public class GetJournalEditServlet extends HttpServlet {
private static final long serialVersionUID = -4635891628211723009L;
private static final Log LOG = LogFactory.getLog(GetJournalEditServlet.class);
static final String STORAGEINFO_PARAM = "storageInfo";
static final String JOURNAL_ID_PARAM = "jid";
static final String SEGMENT_TXID_PARAM = "segmentTxId";
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<String> validRequestors = new HashSet<String>();
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();
}
}

View File

@ -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<Pattern> CURRENT_DIR_PURGE_REGEXES =
ImmutableList.of(
Pattern.compile("edits_\\d+-(\\d+)"),
Pattern.compile("edits_inprogress_(\\d+)(?:\\..*)?"));
private static final List<Pattern> 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 <code>minTxIdToKeep</code> parameter
* are removed.
*/
private static void purgeMatching(File dir, List<Pattern> 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;
}
}

View File

@ -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<EditLogFile> 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<Long> 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<File> localPaths = ImmutableList.of(tmpFile);
LOG.info("Synchronizing log " +
TextFormat.shortDebugString(segment) + " from " + url);
SecurityUtil.doAsLoginUser(
new PrivilegedExceptionAction<Void>() {
@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 &quot;roll forward&quot;
* 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();
}
}
}
}

View File

@ -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 {}
}

View File

@ -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);
}
}
}

View File

@ -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<String, Journal> 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));
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -39,7 +39,8 @@ public final class HdfsServerConstants {
*/
static public enum NodeType {
NAME_NODE,
DATA_NODE;
DATA_NODE,
JOURNAL_NODE;
}
/** Startup options */

View File

@ -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);
}

View File

@ -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();

View File

@ -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<InputStream>() {
@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

View File

@ -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
}
}

View File

@ -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());
}
}

View File

@ -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;

View File

@ -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?
}
}

View File

@ -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<RemoteEditLog> getRemoteEditLogs(long firstTxId) throws IOException {
public List<RemoteEditLog> getRemoteEditLogs(long firstTxId) throws IOException {
File currentDir = sd.getCurrentDir();
List<EditLogFile> allLogFiles = matchEditLogs(currentDir);
List<RemoteEditLog> ret = Lists.newArrayListWithCapacity(
@ -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<EditLogFile> matchEditLogs(File logDir) throws IOException {
public static List<EditLogFile> 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<EditLogInputStream> streams, long fromTxId,
boolean inProgressOk) {
List<EditLogFile> 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<EditLogFile> 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<EditLogFile> getLogFiles(long fromTxId) throws IOException {
public List<EditLogFile> getLogFiles(long fromTxId) throws IOException {
File currentDir = sd.getCurrentDir();
List<EditLogFile> allLogFiles = matchEditLogs(currentDir);
List<EditLogFile> logFiles = Lists.newArrayList();
@ -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<EditLogFile> files = matchEditLogs(dir);
List<EditLogFile> ret = Lists.newLinkedList();
for (EditLogFile elf : files) {
if (elf.getFirstTxId() == startTxId) {
ret.add(elf);
}
}
if (ret.isEmpty()) {
// no matches
return null;
} else if (ret.size() == 1) {
return ret.get(0);
} else {
throw new IllegalStateException("More than one log segment in " +
dir + " starting at txid " + startTxId + ": " +
Joiner.on(", ").join(ret));
}
}
@Override
public String toString() {
@ -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");
}

View File

@ -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<EditLogInputStream> streams,
long fromTxnId, boolean inProgressOk);
long fromTxnId, boolean inProgressOk) throws IOException;
/**
* Set the amount of memory that this stream should use to buffer edits

View File

@ -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<JournalAndStream> 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<JournalAndStream> journals =
new CopyOnWriteArrayList<JournalSet.JournalAndStream>();
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<EditLogInputStream> outStreams,
PriorityQueue<EditLogInputStream> allStreams,
long fromTxId, boolean inProgressOk) {
// We want to group together all the streams that start on the same start
// transaction ID. To do this, we maintain an accumulator (acc) of all
// the streams we've seen at a given start transaction ID. When we see a
@ -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<JournalAndStream> getAllJournalStreams() {
return journals;
}

View File

@ -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<URI> nameDirsToFormat = FSNamesystem.getNamespaceDirs(conf);
List<URI> 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);

View File

@ -41,6 +41,14 @@ final class NameNodeResourcePolicy {
static boolean areResourcesAvailable(
Collection<? extends CheckableNameNodeResource> resources,
int minimumRedundantResources) {
// TODO: workaround:
// - during startup, if there are no edits dirs on disk, then there is
// a call to areResourcesAvailable() with no dirs at all, which was
// previously causing the NN to enter safemode
if (resources.isEmpty()) {
return true;
}
int requiredResourceCount = 0;
int redundantResourceCount = 0;

View File

@ -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("</table></div>\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("<h3> " + nn.getRole() + " Journal Status: </h3>");
out.println("<b>Current transaction ID:</b> " +
nn.getFSImage().getLastAppliedOrWrittenTxId() + "<br/>");
boolean openForWrite = log.isOpenForWrite();
out.println("<div class=\"dfstable\">");
out.println("<table class=\"storage\" title=\"NameNode Journals\">\n"
+ "<thead><tr><td><b>Journal Manager</b></td><td><b>State</b></td></tr></thead>");
for (JournalAndStream jas : log.getJournals()) {
out.print("<tr>");
out.print("<td>" + jas.getManager());
if (jas.isRequired()) {
out.print(" [required]");
}
out.print("</td><td>");
if (jas.isDisabled()) {
out.print("<span class=\"failed\">Failed</span>");
} 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("</td></tr>");
}
out.println("</table></div>");
}
void generateHealthReport(JspWriter out, NameNode nn,
HttpServletRequest request) throws IOException {

View File

@ -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<Void>() {
@Override
public Void run() throws Exception {
try {
doTailEdits();
} catch (InterruptedException e) {
throw new IOException(e);
}
return null;
}
});
}
@VisibleForTesting

View File

@ -17,18 +17,15 @@
*/
package org.apache.hadoop.hdfs.server.protocol;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.io.Writable;
import com.google.common.base.Function;
import com.google.common.collect.ComparisonChain;
public class RemoteEditLog implements Writable, Comparable<RemoteEditLog> {
public class RemoteEditLog implements Comparable<RemoteEditLog> {
private long startTxId = HdfsConstants.INVALID_TXID;
private long endTxId = HdfsConstants.INVALID_TXID;
private boolean isInProgress = false;
public RemoteEditLog() {
}
@ -36,6 +33,13 @@ public class RemoteEditLog implements Writable, Comparable<RemoteEditLog> {
public RemoteEditLog(long startTxId, long endTxId) {
this.startTxId = startTxId;
this.endTxId = endTxId;
this.isInProgress = (endTxId == HdfsConstants.INVALID_TXID);
}
public RemoteEditLog(long startTxId, long endTxId, boolean inProgress) {
this.startTxId = startTxId;
this.endTxId = endTxId;
this.isInProgress = inProgress;
}
public long getStartTxId() {
@ -45,22 +49,18 @@ public class RemoteEditLog implements Writable, Comparable<RemoteEditLog> {
public long getEndTxId() {
return endTxId;
}
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

View File

@ -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);
}
}

View File

@ -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();
}

View File

@ -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 <code>long</code>
* 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();
}
}
}

View File

@ -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;
}

View File

@ -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);
}

View File

@ -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];
}
/**

View File

@ -259,6 +259,11 @@
</description>
</property>
<property>
<name>dfs.namenode.edits.journal-plugin.qjournal</name>
<value>org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager</value>
</property>
<property>
<name>dfs.permissions.enabled</name>
<value>true</value>
@ -1118,4 +1123,21 @@
</description>
</property>
<property>
<name>dfs.journalnode.rpc-address</name>
<value>0.0.0.0:8485</value>
<description>
The JournalNode RPC server address and port.
</description>
</property>
<property>
<name>dfs.journalnode.http-address</name>
<value>0.0.0.0:8480</value>
<description>
The address and port the JournalNode web UI listens on.
If the port is 0 then the server will start on a free port.
</description>
</property>
</configuration>

View File

@ -60,8 +60,10 @@
<%= NamenodeJspHelper.getCorruptFilesWarning(fsn)%>
<% healthjsp.generateHealthReport(out, nn, request); %>
<hr>
<% healthjsp.generateJournalReport(out, nn, request); %>
<hr/>
<% healthjsp.generateConfReport(out, nn, request); %>
<hr>
<%
out.println(ServletUtil.htmlFooter());
%>

View File

@ -0,0 +1,29 @@
<meta HTTP-EQUIV="REFRESH" content="0;url=journalstatus.jsp"/>
<html>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<head><title>Hadoop Administration</title></head>
<body>
<h1>Hadoop Administration</h1>
<ul>
<li><a href="journalstatus.jsp">Status</a></li>
</ul>
</body>
</html>

View File

@ -0,0 +1,42 @@
<%
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
%>
<%@ page
contentType="text/html; charset=UTF-8"
import="org.apache.hadoop.hdfs.server.common.JspHelper"
import="org.apache.hadoop.util.ServletUtil"
%>
<%!
//for java.io.Serializable
private static final long serialVersionUID = 1L;
%>
<!DOCTYPE html>
<html>
<link rel="stylesheet" type="text/css" href="/static/hadoop.css">
<title>Hadoop JournalNode</title>
<body>
<h1>JournalNode</h1>
<%= JspHelper.getVersionTable() %>
<hr />
<br />
<b><a href="/logs/">Logs</a></b>
<%= ServletUtil.htmlFooter() %>

View File

@ -0,0 +1,17 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License. See accompanying LICENSE file.
-->
<web-app version="2.4" xmlns="http://java.sun.com/xml/ns/j2ee">
@journal.servlet.definitions@
</web-app>

View File

@ -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();
}
/**

View File

@ -0,0 +1,194 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.qjournal;
import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.qjournal.server.JournalNode;
import com.google.common.base.Joiner;
import com.google.common.collect.Lists;
public class MiniJournalCluster {
public static class Builder {
private String baseDir;
private int numJournalNodes = 3;
private boolean format = true;
private Configuration conf;
public Builder(Configuration conf) {
this.conf = conf;
}
public Builder baseDir(String d) {
this.baseDir = d;
return this;
}
public Builder numJournalNodes(int n) {
this.numJournalNodes = n;
return this;
}
public Builder format(boolean f) {
this.format = f;
return this;
}
public MiniJournalCluster build() throws IOException {
return new MiniJournalCluster(this);
}
}
private static final Log LOG = LogFactory.getLog(MiniJournalCluster.class);
private File baseDir;
private JournalNode nodes[];
private InetSocketAddress ipcAddrs[];
private InetSocketAddress httpAddrs[];
private MiniJournalCluster(Builder b) throws IOException {
LOG.info("Starting MiniJournalCluster with " +
b.numJournalNodes + " journal nodes");
if (b.baseDir != null) {
this.baseDir = new File(b.baseDir);
} else {
this.baseDir = new File(MiniDFSCluster.getBaseDirectory());
}
nodes = new JournalNode[b.numJournalNodes];
ipcAddrs = new InetSocketAddress[b.numJournalNodes];
httpAddrs = new InetSocketAddress[b.numJournalNodes];
for (int i = 0; i < b.numJournalNodes; i++) {
if (b.format) {
File dir = getStorageDir(i);
LOG.debug("Fully deleting JN directory " + dir);
FileUtil.fullyDelete(dir);
}
nodes[i] = new JournalNode();
nodes[i].setConf(createConfForNode(b, i));
nodes[i].start();
ipcAddrs[i] = nodes[i].getBoundIpcAddress();
httpAddrs[i] = nodes[i].getBoundHttpAddress();
}
}
/**
* Set up the given Configuration object to point to the set of JournalNodes
* in this cluster.
*/
public URI getQuorumJournalURI(String jid) {
List<String> addrs = Lists.newArrayList();
for (InetSocketAddress addr : ipcAddrs) {
addrs.add("127.0.0.1:" + addr.getPort());
}
String addrsVal = Joiner.on(";").join(addrs);
LOG.debug("Setting logger addresses to: " + addrsVal);
try {
return new URI("qjournal://" + addrsVal + "/" + jid);
} catch (URISyntaxException e) {
throw new AssertionError(e);
}
}
/**
* Start the JournalNodes in the cluster.
*/
public void start() throws IOException {
for (JournalNode jn : nodes) {
jn.start();
}
}
/**
* Shutdown all of the JournalNodes in the cluster.
* @throws IOException if one or more nodes failed to stop
*/
public void shutdown() throws IOException {
boolean failed = false;
for (JournalNode jn : nodes) {
try {
jn.stopAndJoin(0);
} catch (Exception e) {
failed = true;
LOG.warn("Unable to stop journal node " + jn, e);
}
}
if (failed) {
throw new IOException("Unable to shut down. Check log for details");
}
}
private Configuration createConfForNode(Builder b, int idx) {
Configuration conf = new Configuration(b.conf);
File logDir = getStorageDir(idx);
conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, logDir.toString());
conf.set(DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_KEY, "0.0.0.0:0");
conf.set(DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
return conf;
}
public File getStorageDir(int idx) {
return new File(baseDir, "journalnode-" + idx).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;
}
}

View File

@ -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<EditLogInputStream> streams,
int firstTxnId, int lastTxnId) throws IOException {
Iterator<EditLogInputStream> 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<EditLogInputStream> 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;
}
}

View File

@ -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();
}
}
}

View File

@ -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();
}
}
}

View File

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

View File

@ -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.<RequestInfo>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.<RequestInfo>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<Boolean>() {
@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.<RequestInfo>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.<RequestInfo>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.<RequestInfo>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.<RequestInfo>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();
}
}

View File

@ -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<Integer> 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<AsyncLogger> 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<Throwable> thrown = new Holder<Throwable>(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<Throwable> 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<Object>(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<Integer, Callable<Void>> 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<Void>() {
@Override
public Void call() throws Exception {
throw new IOException("injected failed IPC at " + idx);
}
});
}
private void inject(int beforeRpcNumber, Callable<Void> injectedCode) {
injections.put(beforeRpcNumber, injectedCode);
}
@Override
protected QJournalProtocol createProxy() throws IOException {
final QJournalProtocol realProxy = super.createProxy();
QJournalProtocol mock = mockProxy(
new WrapEveryCall<Object>(realProxy) {
void beforeCall(InvocationOnMock invocation) throws Exception {
rpcCount++;
String callStr = "[" + addr + "] " +
invocation.getMethod().getName() + "(" +
Joiner.on(", ").join(invocation.getArguments()) + ")";
Callable<Void> 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<Object> wrapper)
throws IOException {
QJournalProtocol mock = Mockito.mock(QJournalProtocol.class,
Mockito.withSettings()
.defaultAnswer(wrapper)
.extraInterfaces(Closeable.class));
return mock;
}
private static abstract class WrapEveryCall<T> implements Answer<T> {
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);
}
}

View File

@ -0,0 +1,69 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.qjournal.client;
import static org.junit.Assert.*;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdfs.qjournal.client.QuorumCall;
import org.junit.Test;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.SettableFuture;
public class TestQuorumCall {
@Test(timeout=10000)
public void testQuorums() throws Exception {
Map<String, SettableFuture<String>> futures = ImmutableMap.of(
"f1", SettableFuture.<String>create(),
"f2", SettableFuture.<String>create(),
"f3", SettableFuture.<String>create());
QuorumCall<String, String> q = QuorumCall.create(futures);
assertEquals(0, q.countResponses());
futures.get("f1").set("first future");
q.waitFor(1, 0, 0, 100000, "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<String, String>(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
}
}
}

View File

@ -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<AsyncLogger> 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<EditLogInputStream> 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<EditLogInputStream> 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<EditLogInputStream> 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.<byte[]>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.<SegmentStateProto>any(), Mockito.<URL>any());
futureThrows(new IOException("mock failure")).when(spies.get(2))
.acceptRecovery(Mockito.<SegmentStateProto>any(), Mockito.<URL>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.<SegmentStateProto>any(), Mockito.<URL>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);
}
}
}

View File

@ -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<AsyncLogger> spyLoggers;
private QuorumJournalManager qjm;
@Before
public void setup() throws Exception {
spyLoggers = ImmutableList.of(
mockLogger(),
mockLogger(),
mockLogger());
qjm = new QuorumJournalManager(conf, new URI("qjournal://host/jid"), FAKE_NSINFO) {
@Override
protected List<AsyncLogger> createLoggers(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.<NamespaceInfo>any());
}
qjm.recoverUnfinalizedSegments();
}
private AsyncLogger mockLogger() {
return Mockito.mock(AsyncLogger.class);
}
static <V> Stubber futureReturns(V value) {
ListenableFuture<V> ret = Futures.immediateFuture(value);
return Mockito.doReturn(ret);
}
static Stubber futureThrows(Throwable t) {
ListenableFuture<?> ret = Futures.immediateFailedFuture(t);
return Mockito.doReturn(ret);
}
@Test
public void testAllLoggersStartOk() throws Exception {
futureReturns(null).when(spyLoggers.get(0)).startLogSegment(Mockito.anyLong());
futureReturns(null).when(spyLoggers.get(1)).startLogSegment(Mockito.anyLong());
futureReturns(null).when(spyLoggers.get(2)).startLogSegment(Mockito.anyLong());
qjm.startLogSegment(1);
}
@Test
public void testQuorumOfLoggersStartOk() throws Exception {
futureReturns(null).when(spyLoggers.get(0)).startLogSegment(Mockito.anyLong());
futureReturns(null).when(spyLoggers.get(1)).startLogSegment(Mockito.anyLong());
futureThrows(new IOException("logger failed"))
.when(spyLoggers.get(2)).startLogSegment(Mockito.anyLong());
qjm.startLogSegment(1);
}
@Test
public void testQuorumOfLoggersFail() throws Exception {
futureReturns(null).when(spyLoggers.get(0)).startLogSegment(Mockito.anyLong());
futureThrows(new IOException("logger failed"))
.when(spyLoggers.get(1)).startLogSegment(Mockito.anyLong());
futureThrows(new IOException("logger failed"))
.when(spyLoggers.get(2)).startLogSegment(Mockito.anyLong());
try {
qjm.startLogSegment(1);
fail("Did not throw when quorum failed");
} catch (QuorumException qe) {
GenericTestUtils.assertExceptionContains("logger failed", qe);
}
}
@Test
public void testWriteEdits() throws Exception {
EditLogOutputStream stm = createLogSegment();
writeOp(stm, 1);
writeOp(stm, 2);
stm.setReadyToFlush();
writeOp(stm, 3);
// The flush should log txn 1-2
futureReturns(null).when(spyLoggers.get(0)).sendEdits(
anyLong(), eq(1L), eq(2), Mockito.<byte[]>any());
futureReturns(null).when(spyLoggers.get(1)).sendEdits(
anyLong(), eq(1L), eq(2), Mockito.<byte[]>any());
futureReturns(null).when(spyLoggers.get(2)).sendEdits(
anyLong(), eq(1L), eq(2), Mockito.<byte[]>any());
stm.flush();
// Another flush should now log txn #3
stm.setReadyToFlush();
futureReturns(null).when(spyLoggers.get(0)).sendEdits(
anyLong(), eq(3L), eq(1), Mockito.<byte[]>any());
futureReturns(null).when(spyLoggers.get(1)).sendEdits(
anyLong(), eq(3L), eq(1), Mockito.<byte[]>any());
futureReturns(null).when(spyLoggers.get(2)).sendEdits(
anyLong(), eq(3L), eq(1), Mockito.<byte[]>any());
stm.flush();
}
@Test
public void testWriteEditsOneSlow() throws Exception {
EditLogOutputStream stm = createLogSegment();
writeOp(stm, 1);
stm.setReadyToFlush();
// Make the first two logs respond immediately
futureReturns(null).when(spyLoggers.get(0)).sendEdits(
anyLong(), eq(1L), eq(1), Mockito.<byte[]>any());
futureReturns(null).when(spyLoggers.get(1)).sendEdits(
anyLong(), eq(1L), eq(1), Mockito.<byte[]>any());
// And the third log not respond
SettableFuture<Void> slowLog = SettableFuture.<Void>create();
Mockito.doReturn(slowLog).when(spyLoggers.get(2)).sendEdits(
anyLong(), eq(1L), eq(1), Mockito.<byte[]>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;
}
}

View File

@ -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<AsyncLogger, PrepareRecoveryResponseProto> makeEntry(
PrepareRecoveryResponseProto proto) {
return Maps.immutableEntry(Mockito.mock(AsyncLogger.class), proto);
}
@Test
public void testComparisons() {
Entry<AsyncLogger, PrepareRecoveryResponseProto> INPROGRESS_1_3 =
makeEntry(PrepareRecoveryResponseProto.newBuilder()
.setSegmentState(SegmentStateProto.newBuilder()
.setStartTxId(1L)
.setEndTxId(3L)
.setIsInProgress(true))
.setLastWriterEpoch(0L)
.build());
Entry<AsyncLogger, PrepareRecoveryResponseProto> INPROGRESS_1_4 =
makeEntry(PrepareRecoveryResponseProto.newBuilder()
.setSegmentState(SegmentStateProto.newBuilder()
.setStartTxId(1L)
.setEndTxId(4L)
.setIsInProgress(true))
.setLastWriterEpoch(0L)
.build());
Entry<AsyncLogger, PrepareRecoveryResponseProto> INPROGRESS_1_4_ACCEPTED =
makeEntry(PrepareRecoveryResponseProto.newBuilder()
.setSegmentState(SegmentStateProto.newBuilder()
.setStartTxId(1L)
.setEndTxId(4L)
.setIsInProgress(true))
.setLastWriterEpoch(0L)
.setAcceptedInEpoch(1L)
.build());
Entry<AsyncLogger, PrepareRecoveryResponseProto> 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));
}
}

View File

@ -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.<File>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);
}
}
}

View File

@ -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
* <em>Acceptor</em> process.
*/
@Test
public void testAcceptRecoveryBehavior() throws Exception {
// We need to run newEpoch() first, or else we have no way to distinguish
// different proposals for the same decision.
try {
ch.prepareRecovery(1L).get();
fail("Did not throw IllegalState when trying to run paxos without an epoch");
} catch (ExecutionException ise) {
GenericTestUtils.assertExceptionContains("bad epoch", ise);
}
ch.newEpoch(1).get();
ch.setEpoch(1);
// prepare() with no previously accepted value and no logs present
PrepareRecoveryResponseProto prep = ch.prepareRecovery(1L).get();
System.err.println("Prep: " + prep);
assertFalse(prep.hasAcceptedInEpoch());
assertFalse(prep.hasSegmentState());
// Make a log segment, and prepare again -- this time should see the
// segment existing.
ch.startLogSegment(1L).get();
ch.sendEdits(1L, 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");
}
}

View File

@ -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<GetReplicaVisibleLengthResponseProto> {
@ -215,8 +215,9 @@ public class TestBlockToken {
tokenGenerationAndVerification(masterHandler, slaveHandler);
}
private Server createMockDatanode(BlockTokenSecretManager sm,
Token<BlockTokenIdentifier> token) throws IOException, ServiceException {
private static Server createMockDatanode(BlockTokenSecretManager sm,
Token<BlockTokenIdentifier> 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<BlockTokenIdentifier> 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<BlockTokenIdentifier> 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);

View File

@ -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.

View File

@ -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);

View File

@ -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());
}

View File

@ -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);

View File

@ -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,

View File

@ -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();
}
}
}

View File

@ -54,7 +54,8 @@
</menu>
<menu name="HDFS" inherit="top">
<item name="High Availability" href="hadoop-yarn/hadoop-yarn-site/HDFSHighAvailability.html"/>
<item name="High Availability With QJM" href="hadoop-yarn/hadoop-yarn-site/HDFSHighAvailabilityWithQJM.html"/>
<item name="High Availability With NFS" href="hadoop-yarn/hadoop-yarn-site/HDFSHighAvailabilityWithNFS.html"/>
<item name="Federation" href="hadoop-yarn/hadoop-yarn-site/Federation.html"/>
<item name="WebHDFS REST API" href="hadoop-yarn/hadoop-yarn-site/WebHDFS.html"/>
<item name="HttpFS Gateway" href="hadoop-hdfs-httpfs/index.html"/>

View File

@ -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
</property>
<property>
<name>dfs.ha.fencing.ssh.connect-timeout</name>
<value>
<value>30000</value>
</property>
---
@ -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 (<hdfs
namenode -format>) 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 <scp> or a similar utility. The location
of the directories containing the NameNode metadata are configured via the
configuration options <<dfs.namenode.name.dir>> and/or
<<dfs.namenode.edits.dir>>. At this time, you should also ensure that the
shared edits dir (as configured by <<dfs.namenode.shared.edits.dir>>) 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 (<hdfs namenode -format>) 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 "<hdfs namenode -bootstrapStandby>" on the
unformatted NameNode. Running this command will also ensure that the shared
edits directory (as configured by <<dfs.namenode.shared.edits.dir>>) 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 "<hdfs -initializeSharedEdits>", 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.
Currently we have the tools to manually recover the data from failed storage nodes.

View File

@ -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 <Active> state,
and the other is in a <Standby> 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:
* <<NameNode machines>> - 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.
* <<JournalNode machines>> - 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. <<Note:>> 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 <<<nameservice ID>>> to identify a
single HDFS instance that may in fact consist of multiple HA NameNodes. In
addition, a new abstraction called <<<NameNode ID>>> 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 <<nameservice ID>> as well as
the <<NameNode ID>>.
** Configuration details
To configure HA NameNodes, you must add several configuration options to your
<<hdfs-site.xml>> configuration file.
The order in which you set these configurations is unimportant, but the values
you choose for <<dfs.nameservices>> and
<<dfs.ha.namenodes.[nameservice ID]>> will determine the keys of those that
follow. Thus, you should decide on these values before setting the rest of the
configuration options.
* <<dfs.nameservices>> - 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.
<<Note:>> 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.
----
<property>
<name>dfs.nameservices</name>
<value>mycluster</value>
</property>
----
* <<dfs.ha.namenodes.[nameservice ID]>> - 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:
----
<property>
<name>dfs.ha.namenodes.mycluster</name>
<value>nn1,nn2</value>
</property>
----
<<Note:>> Currently, only a maximum of two NameNodes may be configured per
nameservice.
* <<dfs.namenode.rpc-address.[nameservice ID].[name node ID]>> - 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:
----
<property>
<name>dfs.namenode.rpc-address.mycluster.nn1</name>
<value>machine1.example.com:8020</value>
</property>
<property>
<name>dfs.namenode.rpc-address.mycluster.nn2</name>
<value>machine2.example.com:8020</value>
</property>
----
<<Note:>> You may similarly configure the "<<servicerpc-address>>" setting if
you so desire.
* <<dfs.namenode.http-address.[nameservice ID].[name node ID]>> - the fully-qualified HTTP address for each NameNode to listen on
Similarly to <rpc-address> above, set the addresses for both NameNodes' HTTP
servers to listen on. For example:
----
<property>
<name>dfs.namenode.http-address.mycluster.nn1</name>
<value>machine1.example.com:50070</value>
</property>
<property>
<name>dfs.namenode.http-address.mycluster.nn2</name>
<value>machine2.example.com:50070</value>
</property>
----
<<Note:>> If you have Hadoop's security features enabled, you should also set
the <https-address> similarly for each NameNode.
* <<dfs.namenode.shared.edits.dir>> - 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,
<<you should only configure one of these URIs.>> The URI should be of the form:
"qjournal://<host1:port1>;<host2:port2>;<host3:port3>/<journalId>". 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):
----
<property>
<name>dfs.namenode.shared.edits.dir</name>
<value>qjournal://node1.example.com:8485;node2.example.com:8485;node3.example.com:8485/mycluster</value>
</property>
----
* <<dfs.client.failover.proxy.provider.[nameservice ID]>> - 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 <<ConfiguredFailoverProxyProvider>>, so use this
unless you are using a custom one. For example:
----
<property>
<name>dfs.client.failover.proxy.provider.mycluster</name>
<value>org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider</value>
</property>
----
* <<dfs.ha.fencing.methods>> - 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. <<Importantly, when using the Quorum
Journal Manager, only one NameNode will ever be allowed to write to the
JournalNodes, so there is no potential for corrupting the file system metadata
from a split-brain scenario.>> 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 "<<<shell(/bin/true)>>>".
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: <shell> and <sshfence>. For information on implementing your own custom
fencing method, see the <org.apache.hadoop.ha.NodeFencer> class.
* <<sshfence>> - SSH to the Active NameNode and kill the process
The <sshfence> option SSHes to the target node and uses <fuser> 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
<<dfs.ha.fencing.ssh.private-key-files>> option, which is a
comma-separated list of SSH private key files. For example:
---
<property>
<name>dfs.ha.fencing.methods</name>
<value>sshfence</value>
</property>
<property>
<name>dfs.ha.fencing.ssh.private-key-files</name>
<value>/home/exampleuser/.ssh/id_rsa</value>
</property>
---
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:
---
<property>
<name>dfs.ha.fencing.methods</name>
<value>sshfence([[username][:port]])</value>
</property>
<property>
<name>dfs.ha.fencing.ssh.connect-timeout</name>
<value>30000</value>
</property>
---
* <<shell>> - run an arbitrary shell command to fence the Active NameNode
The <shell> fencing method runs an arbitrary shell command. It may be
configured like so:
---
<property>
<name>dfs.ha.fencing.methods</name>
<value>shell(/path/to/my/script.sh arg1 arg2 ...)</value>
</property>
---
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
<<dfs_namenode_rpc-address>> will contain the RPC address of the target node, even
though the configuration may specify that variable as
<<dfs.namenode.rpc-address.ns1.nn1>>.
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:
---
<property>
<name>dfs.ha.fencing.methods</name>
<value>shell(/path/to/my/script.sh --nameservice=$target_nameserviceid $target_host:$target_port)</value>
</property>
---
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.
<<Note:>> 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).
* <<fs.defaultFS>> - 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 <<core-site.xml>> file:
---
<property>
<name>fs.defaultFS</name>
<value>hdfs://mycluster</value>
</property>
---
* <<dfs.journalnode.edits.dir>> - 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:
---
<property>
<name>dfs.journalnode.edits.dir</name>
<value>/path/to/journal/node/local/data</value>
</property>
---
** 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 "<hdfs-daemon.sh journalnode>" 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 (<hdfs namenode -format>) 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 "<hdfs namenode -bootstrapStandby>" on the
unformatted NameNode. Running this command will also ensure that the
JournalNodes (as configured by <<dfs.namenode.shared.edits.dir>>) 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 "<hdfs -initializeSharedEdits>", 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 "<hdfs
haadmin>" command. Running this command without any additional arguments will
display the following usage information:
---
Usage: DFSHAAdmin [-ns <nameserviceId>]
[-transitionToActive <serviceId>]
[-transitionToStandby <serviceId>]
[-failover [--forcefence] [--forceactive] <serviceId> <serviceId>]
[-getServiceState <serviceId>]
[-checkHealth <serviceId>]
[-help <command>]
---
This guide describes high-level uses of each of these subcommands. For
specific usage information of each subcommand, you should run "<hdfs haadmin
-help <command>>".
* <<transitionToActive>> and <<transitionToStandby>> - 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. <<These commands do not attempt to perform any fencing,
and thus should rarely be used.>> Instead, one should almost always prefer to
use the "<hdfs haadmin -failover>" subcommand.
* <<failover>> - 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
<<dfs.ha.fencing.methods>>) 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.
* <<getServiceState>> - 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.
* <<checkHealth>> - 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.
<<Note:>> 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:
* <<Failure detection>> - 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.
* <<Active NameNode election>> - 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:
* <<Health monitoring>> - 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.
* <<ZooKeeper session management>> - 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.
* <<ZooKeeper-based election>> - 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 <<<hdfs-site.xml>>> file, add:
----
<property>
<name>dfs.ha.automatic-failover.enabled</name>
<value>true</value>
</property>
----
This specifies that the cluster should be set up for automatic failover.
In your <<<core-site.xml>>> file, add:
----
<property>
<name>ha.zookeeper.quorum</name>
<value>zk1.example.com:2181,zk2.example.com:2181,zk3.example.com:2181</value>
</property>
----
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 <<<dfs.ha.automatic-failover.enabled.my-nameservice-id>>>.
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 <<<start-dfs.sh>>>
Since automatic failover has been enabled in the configuration, the
<<<start-dfs.sh>>> 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 <<<zkfc>>> 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 <<<core-site.xml>>> file:
----
<property>
<name>ha.zookeeper.auth</name>
<value>@/path/to/zk-auth.txt</value>
</property>
<property>
<name>ha.zookeeper.acl</name>
<value>@/path/to/zk-acl.txt</value>
</property>
----
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 <<<hdfs-zkfcs>>> is a unique username for ZooKeeper, and
<<<mypassword>>> 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
<<<zk-acls.txt>>>, prefixed by the string "<<<digest:>>>". For example:
----
digest:hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM=:rwcda
----
In order for these ACLs to take effect, you should then rerun the
<<<zkfc -formatZK>>> 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 <<<kill -9 <pid of NN>>>> 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
<<<ha.zookeeper.session-timeout.ms>>>, but defaults to 5 seconds.
If the test does not succeed, you may have a misconfiguration. Check the logs
for the <<<zkfc>>> daemons as well as the NameNode daemons in order to further
diagnose the issue.
* Automatic Failover FAQ
* <<Is it important that I start the ZKFC and NameNode daemons in any
particular order?>>
No. On any given node you may start the ZKFC before or after its corresponding
NameNode.
* <<What additional monitoring should I put in place?>>
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.
* <<What happens if ZooKeeper goes down?>>
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.
* <<Can I designate one of my NameNodes as primary/preferred?>>
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.
* <<How can I initiate a manual failover when automatic failover is
configured?>>
Even if automatic failover is configured, you may initiate a manual failover
using the same <<<hdfs haadmin>>> command. It will perform a coordinated
failover.