HDFS-1623. High Availability Framework for HDFS NN. Contributed by Todd Lipcon, Aaron T. Myers, Eli Collins, Uma Maheswara Rao G, Bikas Saha, Suresh Srinivas, Jitendra Nath Pandey, Hari Mankude, Brandon Li, Sanjay Radia, Mingjie Lai, and Gregory Chanan

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1296534 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Aaron Myers 2012-03-03 00:42:49 +00:00
commit a1fc055489
269 changed files with 23306 additions and 3301 deletions

View File

@ -0,0 +1,53 @@
Changes for HDFS-1623 branch.
This change list will be merged into the trunk CHANGES.txt when the HDFS-1623
branch is merged.
------------------------------
HADOOP-7455. HA: Introduce HA Service Protocol Interface. (suresh)
HADOOP-7774. HA: Administrative CLI to control HA daemons. (todd)
HADOOP-7896. HA: if both NNs are in Standby mode, client needs to try failing
back and forth several times with sleeps. (atm)
HADOOP-7922. Improve some logging for client IPC failovers and
StandbyExceptions (todd)
HADOOP-7921. StandbyException should extend IOException (todd)
HADOOP-7928. HA: Client failover policy is incorrectly trying to fail over all
IOExceptions (atm)
HADOOP-7925. Add interface and update CLI to query current state to
HAServiceProtocol (eli via todd)
HADOOP-7932. Make client connection retries on socket time outs configurable.
(Uma Maheswara Rao G via todd)
HADOOP-7924. FailoverController for client-based configuration (eli)
HADOOP-7961. Move HA fencing to common. (eli)
HADOOP-7970. HAServiceProtocol methods must throw IOException.
(Hari Mankude via suresh).
HADOOP-7992. Add ZKClient library to facilitate leader election.
(Bikas Saha via suresh).
HADOOP-7983. HA: failover should be able to pass args to fencers. (eli)
HADOOP-7938. HA: the FailoverController should optionally fence the
active during failover. (eli)
HADOOP-7991. HA: the FailoverController should check the standby is
ready before failing over. (eli)
HADOOP-8038. Add 'ipc.client.connect.max.retries.on.timeouts' entry in
core-default.xml file. (Uma Maheswara Rao G via atm)
HADOOP-8041. Log a warning when a failover is first attempted (todd)
HADOOP-8068. void methods can swallow exceptions when going through failover path (todd)
HADOOP-8116. RetriableCommand is using RetryPolicy incorrectly after HADOOP-7896. (atm)

View File

@ -278,8 +278,12 @@
<!-- protobuf generated code -->
<Class name="~org\.apache\.hadoop\.ipc\.protobuf\.ProtocolInfoProtos.*"/>
</Match>
<Match>
<Match>
<!-- protobuf generated code -->
<Class name="~org\.apache\.hadoop\.ipc\.protobuf\.IpcConnectionContextProtos.*"/>
</Match>
<Match>
<!-- protobuf generated code -->
<Class name="~org\.apache\.hadoop\.ha\.proto\.HAServiceProtocolProtos.*"/>
</Match>
</FindBugsFilter>

View File

@ -263,6 +263,38 @@
<artifactId>json-simple</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>com.jcraft</groupId>
<artifactId>jsch</artifactId>
</dependency>
<dependency>
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
<version>3.4.2</version>
<exclusions>
<exclusion>
<!-- otherwise seems to drag in junit 3.8.1 via jline -->
<groupId>junit</groupId>
<artifactId>junit</artifactId>
</exclusion>
<exclusion>
<groupId>com.sun.jdmk</groupId>
<artifactId>jmxtools</artifactId>
</exclusion>
<exclusion>
<groupId>com.sun.jmx</groupId>
<artifactId>jmxri</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
<version>3.4.2</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
</dependencies>
<build>

View File

@ -138,6 +138,12 @@
dfsadmin and mradmin commands to refresh the security policy in-effect.
</td>
</tr>
<tr>
<td><code>security.ha.service.protocol.acl</code></td>
<td>ACL for HAService protocol used by HAAdmin to manage the
active and stand-by states of namenode.
</td>
</tr>
</table>
</section>

View File

@ -114,11 +114,12 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
public static final String
HADOOP_SECURITY_SERVICE_AUTHORIZATION_REFRESH_USER_MAPPINGS =
"security.refresh.user.mappings.protocol.acl";
public static final String
SECURITY_HA_SERVICE_PROTOCOL_ACL = "security.ha.service.protocol.acl";
public static final String HADOOP_SECURITY_TOKEN_SERVICE_USE_IP =
"hadoop.security.token.service.use_ip";
public static final boolean HADOOP_SECURITY_TOKEN_SERVICE_USE_IP_DEFAULT =
true;
}

View File

@ -172,6 +172,11 @@ public class CommonConfigurationKeysPublic {
/** Default value for IPC_CLIENT_CONNECT_MAX_RETRIES_KEY */
public static final int IPC_CLIENT_CONNECT_MAX_RETRIES_DEFAULT = 10;
/** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
public static final String IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY =
"ipc.client.connect.max.retries.on.timeouts";
/** Default value for IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY */
public static final int IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT = 45;
/** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
public static final String IPC_CLIENT_TCPNODELAY_KEY =
"ipc.client.tcpnodelay";
/** Defalt value for IPC_CLIENT_TCPNODELAY_KEY */

View File

@ -0,0 +1,593 @@
/**
* 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.ha;
import java.io.IOException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.AsyncCallback.*;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.KeeperException.Code;
import com.google.common.annotations.VisibleForTesting;
/**
*
* This class implements a simple library to perform leader election on top of
* Apache Zookeeper. Using Zookeeper as a coordination service, leader election
* can be performed by atomically creating an ephemeral lock file (znode) on
* Zookeeper. The service instance that successfully creates the znode becomes
* active and the rest become standbys. <br/>
* This election mechanism is only efficient for small number of election
* candidates (order of 10's) because contention on single znode by a large
* number of candidates can result in Zookeeper overload. <br/>
* The elector does not guarantee fencing (protection of shared resources) among
* service instances. After it has notified an instance about becoming a leader,
* then that instance must ensure that it meets the service consistency
* requirements. If it cannot do so, then it is recommended to quit the
* election. The application implements the {@link ActiveStandbyElectorCallback}
* to interact with the elector
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class ActiveStandbyElector implements Watcher, StringCallback,
StatCallback {
/**
* Callback interface to interact with the ActiveStandbyElector object. <br/>
* The application will be notified with a callback only on state changes
* (i.e. there will never be successive calls to becomeActive without an
* intermediate call to enterNeutralMode). <br/>
* The callbacks will be running on Zookeeper client library threads. The
* application should return from these callbacks quickly so as not to impede
* Zookeeper client library performance and notifications. The app will
* typically remember the state change and return from the callback. It will
* then proceed with implementing actions around that state change. It is
* possible to be called back again while these actions are in flight and the
* app should handle this scenario.
*/
public interface ActiveStandbyElectorCallback {
/**
* This method is called when the app becomes the active leader
*/
void becomeActive();
/**
* This method is called when the app becomes a standby
*/
void becomeStandby();
/**
* If the elector gets disconnected from Zookeeper and does not know about
* the lock state, then it will notify the service via the enterNeutralMode
* interface. The service may choose to ignore this or stop doing state
* changing operations. Upon reconnection, the elector verifies the leader
* status and calls back on the becomeActive and becomeStandby app
* interfaces. <br/>
* Zookeeper disconnects can happen due to network issues or loss of
* Zookeeper quorum. Thus enterNeutralMode can be used to guard against
* split-brain issues. In such situations it might be prudent to call
* becomeStandby too. However, such state change operations might be
* expensive and enterNeutralMode can help guard against doing that for
* transient issues.
*/
void enterNeutralMode();
/**
* If there is any fatal error (e.g. wrong ACL's, unexpected Zookeeper
* errors or Zookeeper persistent unavailability) then notifyFatalError is
* called to notify the app about it.
*/
void notifyFatalError(String errorMessage);
}
/**
* Name of the lock znode used by the library. Protected for access in test
* classes
*/
@VisibleForTesting
protected static final String LOCKFILENAME = "ActiveStandbyElectorLock";
public static final Log LOG = LogFactory.getLog(ActiveStandbyElector.class);
private static final int NUM_RETRIES = 3;
private enum ConnectionState {
DISCONNECTED, CONNECTED, TERMINATED
};
private enum State {
INIT, ACTIVE, STANDBY, NEUTRAL
};
private State state = State.INIT;
private int createRetryCount = 0;
private int statRetryCount = 0;
private ZooKeeper zkClient;
private ConnectionState zkConnectionState = ConnectionState.TERMINATED;
private final ActiveStandbyElectorCallback appClient;
private final String zkHostPort;
private final int zkSessionTimeout;
private final List<ACL> zkAcl;
private byte[] appData;
private final String zkLockFilePath;
private final String znodeWorkingDir;
/**
* Create a new ActiveStandbyElector object <br/>
* The elector is created by providing to it the Zookeeper configuration, the
* parent znode under which to create the znode and a reference to the
* callback interface. <br/>
* The parent znode name must be the same for all service instances and
* different across services. <br/>
* After the leader has been lost, a new leader will be elected after the
* session timeout expires. Hence, the app must set this parameter based on
* its needs for failure response time. The session timeout must be greater
* than the Zookeeper disconnect timeout and is recommended to be 3X that
* value to enable Zookeeper to retry transient disconnections. Setting a very
* short session timeout may result in frequent transitions between active and
* standby states during issues like network outages/GS pauses.
*
* @param zookeeperHostPorts
* ZooKeeper hostPort for all ZooKeeper servers
* @param zookeeperSessionTimeout
* ZooKeeper session timeout
* @param parentZnodeName
* znode under which to create the lock
* @param acl
* ZooKeeper ACL's
* @param app
* reference to callback interface object
* @throws IOException
* @throws HadoopIllegalArgumentException
*/
public ActiveStandbyElector(String zookeeperHostPorts,
int zookeeperSessionTimeout, String parentZnodeName, List<ACL> acl,
ActiveStandbyElectorCallback app) throws IOException,
HadoopIllegalArgumentException {
if (app == null || acl == null || parentZnodeName == null
|| zookeeperHostPorts == null || zookeeperSessionTimeout <= 0) {
throw new HadoopIllegalArgumentException("Invalid argument");
}
zkHostPort = zookeeperHostPorts;
zkSessionTimeout = zookeeperSessionTimeout;
zkAcl = acl;
appClient = app;
znodeWorkingDir = parentZnodeName;
zkLockFilePath = znodeWorkingDir + "/" + LOCKFILENAME;
// createConnection for future API calls
createConnection();
}
/**
* To participate in election, the app will call joinElection. The result will
* be notified by a callback on either the becomeActive or becomeStandby app
* interfaces. <br/>
* After this the elector will automatically monitor the leader status and
* perform re-election if necessary<br/>
* The app could potentially start off in standby mode and ignore the
* becomeStandby call.
*
* @param data
* to be set by the app. non-null data must be set.
* @throws HadoopIllegalArgumentException
* if valid data is not supplied
*/
public synchronized void joinElection(byte[] data)
throws HadoopIllegalArgumentException {
LOG.debug("Attempting active election");
if (data == null) {
throw new HadoopIllegalArgumentException("data cannot be null");
}
appData = new byte[data.length];
System.arraycopy(data, 0, appData, 0, data.length);
joinElectionInternal();
}
/**
* Any service instance can drop out of the election by calling quitElection.
* <br/>
* This will lose any leader status, if held, and stop monitoring of the lock
* node. <br/>
* If the instance wants to participate in election again, then it needs to
* call joinElection(). <br/>
* This allows service instances to take themselves out of rotation for known
* impending unavailable states (e.g. long GC pause or software upgrade).
*/
public synchronized void quitElection() {
LOG.debug("Yielding from election");
reset();
}
/**
* Exception thrown when there is no active leader
*/
public static class ActiveNotFoundException extends Exception {
private static final long serialVersionUID = 3505396722342846462L;
}
/**
* get data set by the active leader
*
* @return data set by the active instance
* @throws ActiveNotFoundException
* when there is no active leader
* @throws KeeperException
* other zookeeper operation errors
* @throws InterruptedException
* @throws IOException
* when ZooKeeper connection could not be established
*/
public synchronized byte[] getActiveData() throws ActiveNotFoundException,
KeeperException, InterruptedException, IOException {
try {
if (zkClient == null) {
createConnection();
}
Stat stat = new Stat();
return zkClient.getData(zkLockFilePath, false, stat);
} catch(KeeperException e) {
Code code = e.code();
if (operationNodeDoesNotExist(code)) {
// handle the commonly expected cases that make sense for us
throw new ActiveNotFoundException();
} else {
throw e;
}
}
}
/**
* interface implementation of Zookeeper callback for create
*/
@Override
public synchronized void processResult(int rc, String path, Object ctx,
String name) {
LOG.debug("CreateNode result: " + rc + " for path: " + path
+ " connectionState: " + zkConnectionState);
if (zkClient == null) {
// zkClient is nulled before closing the connection
// this is the callback with session expired after we closed the session
return;
}
Code code = Code.get(rc);
if (operationSuccess(code)) {
// we successfully created the znode. we are the leader. start monitoring
becomeActive();
monitorActiveStatus();
return;
}
if (operationNodeExists(code)) {
if (createRetryCount == 0) {
// znode exists and we did not retry the operation. so a different
// instance has created it. become standby and monitor lock.
becomeStandby();
}
// if we had retried then the znode could have been created by our first
// attempt to the server (that we lost) and this node exists response is
// for the second attempt. verify this case via ephemeral node owner. this
// will happen on the callback for monitoring the lock.
monitorActiveStatus();
return;
}
String errorMessage = "Received create error from Zookeeper. code:"
+ code.toString();
LOG.debug(errorMessage);
if (operationRetry(code)) {
if (createRetryCount < NUM_RETRIES) {
LOG.debug("Retrying createNode createRetryCount: " + createRetryCount);
++createRetryCount;
createNode();
return;
}
errorMessage = errorMessage
+ ". Not retrying further znode create connection errors.";
}
fatalError(errorMessage);
}
/**
* interface implementation of Zookeeper callback for monitor (exists)
*/
@Override
public synchronized void processResult(int rc, String path, Object ctx,
Stat stat) {
LOG.debug("StatNode result: " + rc + " for path: " + path
+ " connectionState: " + zkConnectionState);
if (zkClient == null) {
// zkClient is nulled before closing the connection
// this is the callback with session expired after we closed the session
return;
}
Code code = Code.get(rc);
if (operationSuccess(code)) {
// the following owner check completes verification in case the lock znode
// creation was retried
if (stat.getEphemeralOwner() == zkClient.getSessionId()) {
// we own the lock znode. so we are the leader
becomeActive();
} else {
// we dont own the lock znode. so we are a standby.
becomeStandby();
}
// the watch set by us will notify about changes
return;
}
if (operationNodeDoesNotExist(code)) {
// the lock znode disappeared before we started monitoring it
enterNeutralMode();
joinElectionInternal();
return;
}
String errorMessage = "Received stat error from Zookeeper. code:"
+ code.toString();
LOG.debug(errorMessage);
if (operationRetry(code)) {
if (statRetryCount < NUM_RETRIES) {
++statRetryCount;
monitorNode();
return;
}
errorMessage = errorMessage
+ ". Not retrying further znode monitoring connection errors.";
}
fatalError(errorMessage);
}
/**
* interface implementation of Zookeeper watch events (connection and node)
*/
@Override
public synchronized void process(WatchedEvent event) {
Event.EventType eventType = event.getType();
LOG.debug("Watcher event type: " + eventType + " with state:"
+ event.getState() + " for path:" + event.getPath()
+ " connectionState: " + zkConnectionState);
if (zkClient == null) {
// zkClient is nulled before closing the connection
// this is the callback with session expired after we closed the session
return;
}
if (eventType == Event.EventType.None) {
// the connection state has changed
switch (event.getState()) {
case SyncConnected:
// if the listener was asked to move to safe state then it needs to
// be undone
ConnectionState prevConnectionState = zkConnectionState;
zkConnectionState = ConnectionState.CONNECTED;
if (prevConnectionState == ConnectionState.DISCONNECTED) {
monitorActiveStatus();
}
break;
case Disconnected:
// ask the app to move to safe state because zookeeper connection
// is not active and we dont know our state
zkConnectionState = ConnectionState.DISCONNECTED;
enterNeutralMode();
break;
case Expired:
// the connection got terminated because of session timeout
// call listener to reconnect
enterNeutralMode();
reJoinElection();
break;
default:
fatalError("Unexpected Zookeeper watch event state: "
+ event.getState());
break;
}
return;
}
// a watch on lock path in zookeeper has fired. so something has changed on
// the lock. ideally we should check that the path is the same as the lock
// path but trusting zookeeper for now
String path = event.getPath();
if (path != null) {
switch (eventType) {
case NodeDeleted:
if (state == State.ACTIVE) {
enterNeutralMode();
}
joinElectionInternal();
break;
case NodeDataChanged:
monitorActiveStatus();
break;
default:
LOG.debug("Unexpected node event: " + eventType + " for path: " + path);
monitorActiveStatus();
}
return;
}
// some unexpected error has occurred
fatalError("Unexpected watch error from Zookeeper");
}
/**
* Get a new zookeeper client instance. protected so that test class can
* inherit and pass in a mock object for zookeeper
*
* @return new zookeeper client instance
* @throws IOException
*/
protected synchronized ZooKeeper getNewZooKeeper() throws IOException {
return new ZooKeeper(zkHostPort, zkSessionTimeout, this);
}
private void fatalError(String errorMessage) {
reset();
appClient.notifyFatalError(errorMessage);
}
private void monitorActiveStatus() {
LOG.debug("Monitoring active leader");
statRetryCount = 0;
monitorNode();
}
private void joinElectionInternal() {
if (zkClient == null) {
if (!reEstablishSession()) {
fatalError("Failed to reEstablish connection with ZooKeeper");
return;
}
}
createRetryCount = 0;
createNode();
}
private void reJoinElection() {
LOG.debug("Trying to re-establish ZK session");
terminateConnection();
joinElectionInternal();
}
private boolean reEstablishSession() {
int connectionRetryCount = 0;
boolean success = false;
while(!success && connectionRetryCount < NUM_RETRIES) {
LOG.debug("Establishing zookeeper connection");
try {
createConnection();
success = true;
} catch(IOException e) {
LOG.warn(e);
try {
Thread.sleep(5000);
} catch(InterruptedException e1) {
LOG.warn(e1);
}
}
++connectionRetryCount;
}
return success;
}
private void createConnection() throws IOException {
zkClient = getNewZooKeeper();
}
private void terminateConnection() {
if (zkClient == null) {
return;
}
LOG.debug("Terminating ZK connection");
ZooKeeper tempZk = zkClient;
zkClient = null;
try {
tempZk.close();
} catch(InterruptedException e) {
LOG.warn(e);
}
zkConnectionState = ConnectionState.TERMINATED;
}
private void reset() {
state = State.INIT;
terminateConnection();
}
private void becomeActive() {
if (state != State.ACTIVE) {
LOG.debug("Becoming active");
state = State.ACTIVE;
appClient.becomeActive();
}
}
private void becomeStandby() {
if (state != State.STANDBY) {
LOG.debug("Becoming standby");
state = State.STANDBY;
appClient.becomeStandby();
}
}
private void enterNeutralMode() {
if (state != State.NEUTRAL) {
LOG.debug("Entering neutral mode");
state = State.NEUTRAL;
appClient.enterNeutralMode();
}
}
private void createNode() {
zkClient.create(zkLockFilePath, appData, zkAcl, CreateMode.EPHEMERAL, this,
null);
}
private void monitorNode() {
zkClient.exists(zkLockFilePath, true, this, null);
}
private boolean operationSuccess(Code code) {
return (code == Code.OK);
}
private boolean operationNodeExists(Code code) {
return (code == Code.NODEEXISTS);
}
private boolean operationNodeDoesNotExist(Code code) {
return (code == Code.NONODE);
}
private boolean operationRetry(Code code) {
switch (code) {
case CONNECTIONLOSS:
case OPERATIONTIMEOUT:
return true;
}
return false;
}
}

View File

@ -0,0 +1,36 @@
/**
* 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.ha;
import java.io.IOException;
/**
* Indicates that the operator has specified an invalid configuration
* for fencing methods.
*/
class BadFencingConfigurationException extends IOException {
private static final long serialVersionUID = 1L;
public BadFencingConfigurationException(String msg) {
super(msg);
}
public BadFencingConfigurationException(String msg, Throwable cause) {
super(msg, cause);
}
}

View File

@ -0,0 +1,184 @@
/**
* 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.ha;
import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import com.google.common.base.Preconditions;
/**
* The FailOverController is responsible for electing an active service
* on startup or when the current active is changing (eg due to failure),
* monitoring the health of a service, and performing a fail-over when a
* new active service is either manually selected by a user or elected.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class FailoverController {
private static final Log LOG = LogFactory.getLog(FailoverController.class);
/**
* Perform pre-failover checks on the given service we plan to
* failover to, eg to prevent failing over to a service (eg due
* to it being inaccessible, already active, not healthy, etc).
*
* An option to ignore toSvc if it claims it is not ready to
* become active is provided in case performing a failover will
* allow it to become active, eg because it triggers a log roll
* so the standby can learn about new blocks and leave safemode.
*
* @param toSvc service to make active
* @param toSvcName name of service to make active
* @param forceActive ignore toSvc if it reports that it is not ready
* @throws FailoverFailedException if we should avoid failover
*/
private static void preFailoverChecks(HAServiceProtocol toSvc,
InetSocketAddress toSvcAddr,
boolean forceActive)
throws FailoverFailedException {
HAServiceState toSvcState;
try {
toSvcState = toSvc.getServiceState();
} catch (IOException e) {
String msg = "Unable to get service state for " + toSvcAddr;
LOG.error(msg, e);
throw new FailoverFailedException(msg, e);
}
if (!toSvcState.equals(HAServiceState.STANDBY)) {
throw new FailoverFailedException(
"Can't failover to an active service");
}
try {
HAServiceProtocolHelper.monitorHealth(toSvc);
} catch (HealthCheckFailedException hce) {
throw new FailoverFailedException(
"Can't failover to an unhealthy service", hce);
} catch (IOException e) {
throw new FailoverFailedException(
"Got an IO exception", e);
}
try {
if (!toSvc.readyToBecomeActive()) {
if (!forceActive) {
throw new FailoverFailedException(
toSvcAddr + " is not ready to become active");
}
}
} catch (IOException e) {
throw new FailoverFailedException(
"Got an IO exception", e);
}
}
/**
* Failover from service 1 to service 2. If the failover fails
* then try to failback.
*
* @param fromSvc currently active service
* @param fromSvcAddr addr of the currently active service
* @param toSvc service to make active
* @param toSvcAddr addr of the service to make active
* @param fencer for fencing fromSvc
* @param forceFence to fence fromSvc even if not strictly necessary
* @param forceActive try to make toSvc active even if it is not ready
* @throws FailoverFailedException if the failover fails
*/
public static void failover(HAServiceProtocol fromSvc,
InetSocketAddress fromSvcAddr,
HAServiceProtocol toSvc,
InetSocketAddress toSvcAddr,
NodeFencer fencer,
boolean forceFence,
boolean forceActive)
throws FailoverFailedException {
Preconditions.checkArgument(fencer != null, "failover requires a fencer");
preFailoverChecks(toSvc, toSvcAddr, forceActive);
// Try to make fromSvc standby
boolean tryFence = true;
try {
HAServiceProtocolHelper.transitionToStandby(fromSvc);
// We should try to fence if we failed or it was forced
tryFence = forceFence ? true : false;
} catch (ServiceFailedException sfe) {
LOG.warn("Unable to make " + fromSvcAddr + " standby (" +
sfe.getMessage() + ")");
} catch (IOException ioe) {
LOG.warn("Unable to make " + fromSvcAddr +
" standby (unable to connect)", ioe);
}
// Fence fromSvc if it's required or forced by the user
if (tryFence) {
if (!fencer.fence(fromSvcAddr)) {
throw new FailoverFailedException("Unable to fence " +
fromSvcAddr + ". Fencing failed.");
}
}
// Try to make toSvc active
boolean failed = false;
Throwable cause = null;
try {
HAServiceProtocolHelper.transitionToActive(toSvc);
} catch (ServiceFailedException sfe) {
LOG.error("Unable to make " + toSvcAddr + " active (" +
sfe.getMessage() + "). Failing back.");
failed = true;
cause = sfe;
} catch (IOException ioe) {
LOG.error("Unable to make " + toSvcAddr +
" active (unable to connect). Failing back.", ioe);
failed = true;
cause = ioe;
}
// We failed to make toSvc active
if (failed) {
String msg = "Unable to failover to " + toSvcAddr;
// Only try to failback if we didn't fence fromSvc
if (!tryFence) {
try {
// Unconditionally fence toSvc in case it is still trying to
// become active, eg we timed out waiting for its response.
// Unconditionally force fromSvc to become active since it
// was previously active when we initiated failover.
failover(toSvc, toSvcAddr, fromSvc, fromSvcAddr, fencer, true, true);
} catch (FailoverFailedException ffe) {
msg += ". Failback to " + fromSvcAddr +
" failed (" + ffe.getMessage() + ")";
LOG.fatal(msg);
}
}
throw new FailoverFailedException(msg, cause);
}
}
}

View File

@ -0,0 +1,38 @@
/**
* 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.ha;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Exception thrown to indicate service failover has failed.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class FailoverFailedException extends Exception {
private static final long serialVersionUID = 1L;
public FailoverFailedException(final String message) {
super(message);
}
public FailoverFailedException(String message, Throwable cause) {
super(message, cause);
}
}

View File

@ -0,0 +1,67 @@
/**
* 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.ha;
import java.net.InetSocketAddress;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configurable;
/**
* A fencing method is a method by which one node can forcibly prevent
* another node from making continued progress. This might be implemented
* by killing a process on the other node, by denying the other node's
* access to shared storage, or by accessing a PDU to cut the other node's
* power.
* <p>
* Since these methods are often vendor- or device-specific, operators
* may implement this interface in order to achieve fencing.
* <p>
* Fencing is configured by the operator as an ordered list of methods to
* attempt. Each method will be tried in turn, and the next in the list
* will only be attempted if the previous one fails. See {@link NodeFencer}
* for more information.
* <p>
* If an implementation also implements {@link Configurable} then its
* <code>setConf</code> method will be called upon instantiation.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public interface FenceMethod {
/**
* Verify that the given fencing method's arguments are valid.
* @param args the arguments provided in the configuration. This may
* be null if the operator did not configure any arguments.
* @throws BadFencingConfigurationException if the arguments are invalid
*/
public void checkArgs(String args) throws BadFencingConfigurationException;
/**
* Attempt to fence the target node.
* @param serviceAddr the address (host:ipcport) of the service to fence
* @param args the configured arguments, which were checked at startup by
* {@link #checkArgs(String)}
* @return true if fencing was successful, false if unsuccessful or
* indeterminate
* @throws BadFencingConfigurationException if the configuration was
* determined to be invalid only at runtime
*/
public boolean tryFence(InetSocketAddress serviceAddr, String args)
throws BadFencingConfigurationException;
}

View File

@ -0,0 +1,321 @@
/**
* 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.ha;
import java.io.IOException;
import java.io.PrintStream;
import java.net.InetSocketAddress;
import java.util.Map;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.GnuParser;
import org.apache.commons.cli.ParseException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.ha.protocolPB.HAServiceProtocolClientSideTranslatorPB;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import com.google.common.collect.ImmutableMap;
/**
* A command-line tool for making calls in the HAServiceProtocol.
* For example,. this can be used to force a service to standby or active
* mode, or to trigger a health-check.
*/
@InterfaceAudience.Private
public abstract class HAAdmin extends Configured implements Tool {
private static final String FORCEFENCE = "forcefence";
private static final String FORCEACTIVE = "forceactive";
private static Map<String, UsageInfo> USAGE =
ImmutableMap.<String, UsageInfo>builder()
.put("-transitionToActive",
new UsageInfo("<serviceId>", "Transitions the service into Active state"))
.put("-transitionToStandby",
new UsageInfo("<serviceId>", "Transitions the service into Standby state"))
.put("-failover",
new UsageInfo("[--"+FORCEFENCE+"] [--"+FORCEACTIVE+"] <serviceId> <serviceId>",
"Failover from the first service to the second.\n" +
"Unconditionally fence services if the "+FORCEFENCE+" option is used.\n" +
"Try to failover to the target service even if it is not ready if the " +
FORCEACTIVE + " option is used."))
.put("-getServiceState",
new UsageInfo("<serviceId>", "Returns the state of the service"))
.put("-checkHealth",
new UsageInfo("<serviceId>",
"Requests that the service perform a health check.\n" +
"The HAAdmin tool will exit with a non-zero exit code\n" +
"if the check fails."))
.put("-help",
new UsageInfo("<command>", "Displays help on the specified command"))
.build();
/** Output stream for errors, for use in tests */
protected PrintStream errOut = System.err;
PrintStream out = System.out;
protected String getUsageString() {
return "Usage: HAAdmin";
}
protected void printUsage(PrintStream errOut) {
errOut.println(getUsageString());
for (Map.Entry<String, UsageInfo> e : USAGE.entrySet()) {
String cmd = e.getKey();
UsageInfo usage = e.getValue();
errOut.println(" [" + cmd + " " + usage.args + "]");
}
errOut.println();
ToolRunner.printGenericCommandUsage(errOut);
}
private static void printUsage(PrintStream errOut, String cmd) {
UsageInfo usage = USAGE.get(cmd);
if (usage == null) {
throw new RuntimeException("No usage for cmd " + cmd);
}
errOut.println("Usage: HAAdmin [" + cmd + " " + usage.args + "]");
}
private int transitionToActive(final String[] argv)
throws IOException, ServiceFailedException {
if (argv.length != 2) {
errOut.println("transitionToActive: incorrect number of arguments");
printUsage(errOut, "-transitionToActive");
return -1;
}
HAServiceProtocol proto = getProtocol(argv[1]);
HAServiceProtocolHelper.transitionToActive(proto);
return 0;
}
private int transitionToStandby(final String[] argv)
throws IOException, ServiceFailedException {
if (argv.length != 2) {
errOut.println("transitionToStandby: incorrect number of arguments");
printUsage(errOut, "-transitionToStandby");
return -1;
}
HAServiceProtocol proto = getProtocol(argv[1]);
HAServiceProtocolHelper.transitionToStandby(proto);
return 0;
}
private int failover(final String[] argv)
throws IOException, ServiceFailedException {
Configuration conf = getConf();
boolean forceFence = false;
boolean forceActive = false;
Options failoverOpts = new Options();
// "-failover" isn't really an option but we need to add
// it to appease CommandLineParser
failoverOpts.addOption("failover", false, "failover");
failoverOpts.addOption(FORCEFENCE, false, "force fencing");
failoverOpts.addOption(FORCEACTIVE, false, "force failover");
CommandLineParser parser = new GnuParser();
CommandLine cmd;
try {
cmd = parser.parse(failoverOpts, argv);
forceFence = cmd.hasOption(FORCEFENCE);
forceActive = cmd.hasOption(FORCEACTIVE);
} catch (ParseException pe) {
errOut.println("failover: incorrect arguments");
printUsage(errOut, "-failover");
return -1;
}
int numOpts = cmd.getOptions() == null ? 0 : cmd.getOptions().length;
final String[] args = cmd.getArgs();
if (numOpts > 2 || args.length != 2) {
errOut.println("failover: incorrect arguments");
printUsage(errOut, "-failover");
return -1;
}
NodeFencer fencer;
try {
fencer = NodeFencer.create(conf);
} catch (BadFencingConfigurationException bfce) {
errOut.println("failover: incorrect fencing configuration: " +
bfce.getLocalizedMessage());
return -1;
}
if (fencer == null) {
errOut.println("failover: no fencer configured");
return -1;
}
InetSocketAddress addr1 =
NetUtils.createSocketAddr(getServiceAddr(args[0]));
InetSocketAddress addr2 =
NetUtils.createSocketAddr(getServiceAddr(args[1]));
HAServiceProtocol proto1 = getProtocol(args[0]);
HAServiceProtocol proto2 = getProtocol(args[1]);
try {
FailoverController.failover(proto1, addr1, proto2, addr2,
fencer, forceFence, forceActive);
out.println("Failover from "+args[0]+" to "+args[1]+" successful");
} catch (FailoverFailedException ffe) {
errOut.println("Failover failed: " + ffe.getLocalizedMessage());
return -1;
}
return 0;
}
private int checkHealth(final String[] argv)
throws IOException, ServiceFailedException {
if (argv.length != 2) {
errOut.println("checkHealth: incorrect number of arguments");
printUsage(errOut, "-checkHealth");
return -1;
}
HAServiceProtocol proto = getProtocol(argv[1]);
try {
HAServiceProtocolHelper.monitorHealth(proto);
} catch (HealthCheckFailedException e) {
errOut.println("Health check failed: " + e.getLocalizedMessage());
return -1;
}
return 0;
}
private int getServiceState(final String[] argv)
throws IOException, ServiceFailedException {
if (argv.length != 2) {
errOut.println("getServiceState: incorrect number of arguments");
printUsage(errOut, "-getServiceState");
return -1;
}
HAServiceProtocol proto = getProtocol(argv[1]);
out.println(proto.getServiceState());
return 0;
}
/**
* Return the serviceId as is, we are assuming it was
* given as a service address of form <host:ipcport>.
*/
protected String getServiceAddr(String serviceId) {
return serviceId;
}
/**
* Return a proxy to the specified target service.
*/
protected HAServiceProtocol getProtocol(String serviceId)
throws IOException {
String serviceAddr = getServiceAddr(serviceId);
InetSocketAddress addr = NetUtils.createSocketAddr(serviceAddr);
return new HAServiceProtocolClientSideTranslatorPB(addr, getConf());
}
@Override
public int run(String[] argv) throws Exception {
try {
return runCmd(argv);
} catch (IllegalArgumentException iae) {
errOut.println("Illegal argument: " + iae.getLocalizedMessage());
return -1;
} catch (IOException ioe) {
errOut.println("Operation failed: " + ioe.getLocalizedMessage());
return -1;
}
}
protected int runCmd(String[] argv) throws Exception {
if (argv.length < 1) {
printUsage(errOut);
return -1;
}
String cmd = argv[0];
if (!cmd.startsWith("-")) {
errOut.println("Bad command '" + cmd + "': expected command starting with '-'");
printUsage(errOut);
return -1;
}
if ("-transitionToActive".equals(cmd)) {
return transitionToActive(argv);
} else if ("-transitionToStandby".equals(cmd)) {
return transitionToStandby(argv);
} else if ("-failover".equals(cmd)) {
return failover(argv);
} else if ("-getServiceState".equals(cmd)) {
return getServiceState(argv);
} else if ("-checkHealth".equals(cmd)) {
return checkHealth(argv);
} else if ("-help".equals(cmd)) {
return help(argv);
} else {
errOut.println(cmd.substring(1) + ": Unknown command");
printUsage(errOut);
return -1;
}
}
private int help(String[] argv) {
if (argv.length != 2) {
printUsage(errOut, "-help");
return -1;
}
String cmd = argv[1];
if (!cmd.startsWith("-")) {
cmd = "-" + cmd;
}
UsageInfo usageInfo = USAGE.get(cmd);
if (usageInfo == null) {
errOut.println(cmd + ": Unknown command");
printUsage(errOut);
return -1;
}
errOut.println(cmd + " [" + usageInfo.args + "]: " + usageInfo.help);
return 0;
}
private static class UsageInfo {
private final String args;
private final String help;
public UsageInfo(String args, String help) {
this.args = args;
this.help = help;
}
}
}

View File

@ -0,0 +1,141 @@
/**
* 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.ha;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.KerberosInfo;
import java.io.IOException;
/**
* Protocol interface that provides High Availability related primitives to
* monitor and fail-over the service.
*
* This interface could be used by HA frameworks to manage the service.
*/
@KerberosInfo(
serverPrincipal=CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface HAServiceProtocol {
/**
* Initial version of the protocol
*/
public static final long versionID = 1L;
/**
* An HA service may be in active or standby state. During
* startup, it is in an unknown INITIALIZING state.
*/
public enum HAServiceState {
INITIALIZING("initializing"),
ACTIVE("active"),
STANDBY("standby");
private String name;
HAServiceState(String name) {
this.name = name;
}
public String toString() {
return name;
}
}
/**
* Monitor the health of service. This periodically called by the HA
* frameworks to monitor the health of the service.
*
* Service is expected to perform checks to ensure it is functional.
* If the service is not healthy due to failure or partial failure,
* it is expected to throw {@link HealthCheckFailedException}.
* The definition of service not healthy is left to the service.
*
* Note that when health check of an Active service fails,
* failover to standby may be done.
*
* @throws HealthCheckFailedException
* if the health check of a service fails.
* @throws AccessControlException
* if access is denied.
* @throws IOException
* if other errors happen
*/
public void monitorHealth() throws HealthCheckFailedException,
AccessControlException,
IOException;
/**
* Request service to transition to active state. No operation, if the
* service is already in active state.
*
* @throws ServiceFailedException
* if transition from standby to active fails.
* @throws AccessControlException
* if access is denied.
* @throws IOException
* if other errors happen
*/
public void transitionToActive() throws ServiceFailedException,
AccessControlException,
IOException;
/**
* Request service to transition to standby state. No operation, if the
* service is already in standby state.
*
* @throws ServiceFailedException
* if transition from active to standby fails.
* @throws AccessControlException
* if access is denied.
* @throws IOException
* if other errors happen
*/
public void transitionToStandby() throws ServiceFailedException,
AccessControlException,
IOException;
/**
* Return the current state of the service.
*
* @throws AccessControlException
* if access is denied.
* @throws IOException
* if other errors happen
*/
public HAServiceState getServiceState() throws AccessControlException,
IOException;
/**
* Return true if the service is capable and ready to transition
* from the standby state to the active state.
*
* @return true if the service is ready to become active, false otherwise.
* @throws AccessControlException
* if access is denied.
* @throws IOException
* if other errors happen
*/
public boolean readyToBecomeActive() throws ServiceFailedException,
AccessControlException,
IOException;
}

View File

@ -0,0 +1,59 @@
/**
* 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.ha;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.ipc.RemoteException;
/**
* Helper for making {@link HAServiceProtocol} RPC calls. This helper
* unwraps the {@link RemoteException} to specific exceptions.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class HAServiceProtocolHelper {
public static void monitorHealth(HAServiceProtocol svc)
throws IOException {
try {
svc.monitorHealth();
} catch (RemoteException e) {
throw e.unwrapRemoteException(HealthCheckFailedException.class);
}
}
public static void transitionToActive(HAServiceProtocol svc)
throws IOException {
try {
svc.transitionToActive();
} catch (RemoteException e) {
throw e.unwrapRemoteException(ServiceFailedException.class);
}
}
public static void transitionToStandby(HAServiceProtocol svc)
throws IOException {
try {
svc.transitionToStandby();
} catch (RemoteException e) {
throw e.unwrapRemoteException(ServiceFailedException.class);
}
}
}

View File

@ -0,0 +1,40 @@
/**
* 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.ha;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Exception thrown to indicate that health check of a service failed.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class HealthCheckFailedException extends IOException {
private static final long serialVersionUID = 1L;
public HealthCheckFailedException(final String message) {
super(message);
}
public HealthCheckFailedException(String message, Throwable cause) {
super(message, cause);
}
}

View File

@ -0,0 +1,195 @@
/**
* 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.ha;
import java.net.InetSocketAddress;
import java.util.List;
import java.util.Map;
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.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.ReflectionUtils;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
/**
* This class parses the configured list of fencing methods, and
* is responsible for trying each one in turn while logging informative
* output.<p>
*
* The fencing methods are configured as a carriage-return separated list.
* Each line in the list is of the form:<p>
* <code>com.example.foo.MyMethod(arg string)</code>
* or
* <code>com.example.foo.MyMethod</code>
* The class provided must implement the {@link FenceMethod} interface.
* The fencing methods that ship with Hadoop may also be referred to
* by shortened names:<p>
* <ul>
* <li><code>shell(/path/to/some/script.sh args...)</code></li>
* <li><code>sshfence(...)</code> (see {@link SshFenceByTcpPort})
* </ul>
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class NodeFencer {
public static final String CONF_METHODS_KEY =
"dfs.ha.fencing.methods";
private static final String CLASS_RE = "([a-zA-Z0-9\\.\\$]+)";
private static final Pattern CLASS_WITH_ARGUMENT =
Pattern.compile(CLASS_RE + "\\((.+?)\\)");
private static final Pattern CLASS_WITHOUT_ARGUMENT =
Pattern.compile(CLASS_RE);
private static final Pattern HASH_COMMENT_RE =
Pattern.compile("#.*$");
private static final Log LOG = LogFactory.getLog(NodeFencer.class);
/**
* Standard fencing methods included with Hadoop.
*/
private static final Map<String, Class<? extends FenceMethod>> STANDARD_METHODS =
ImmutableMap.<String, Class<? extends FenceMethod>>of(
"shell", ShellCommandFencer.class,
"sshfence", SshFenceByTcpPort.class);
private final List<FenceMethodWithArg> methods;
public NodeFencer(Configuration conf)
throws BadFencingConfigurationException {
this.methods = parseMethods(conf);
}
public static NodeFencer create(Configuration conf)
throws BadFencingConfigurationException {
String confStr = conf.get(CONF_METHODS_KEY);
if (confStr == null) {
return null;
}
return new NodeFencer(conf);
}
public boolean fence(InetSocketAddress serviceAddr) {
LOG.info("====== Beginning Service Fencing Process... ======");
int i = 0;
for (FenceMethodWithArg method : methods) {
LOG.info("Trying method " + (++i) + "/" + methods.size() +": " + method);
try {
if (method.method.tryFence(serviceAddr, method.arg)) {
LOG.info("====== Fencing successful by method " + method + " ======");
return true;
}
} catch (BadFencingConfigurationException e) {
LOG.error("Fencing method " + method + " misconfigured", e);
continue;
} catch (Throwable t) {
LOG.error("Fencing method " + method + " failed with an unexpected error.", t);
continue;
}
LOG.warn("Fencing method " + method + " was unsuccessful.");
}
LOG.error("Unable to fence service by any configured method.");
return false;
}
private static List<FenceMethodWithArg> parseMethods(Configuration conf)
throws BadFencingConfigurationException {
String confStr = conf.get(CONF_METHODS_KEY);
String[] lines = confStr.split("\\s*\n\\s*");
List<FenceMethodWithArg> methods = Lists.newArrayList();
for (String line : lines) {
line = HASH_COMMENT_RE.matcher(line).replaceAll("");
line = line.trim();
if (!line.isEmpty()) {
methods.add(parseMethod(conf, line));
}
}
return methods;
}
private static FenceMethodWithArg parseMethod(Configuration conf, String line)
throws BadFencingConfigurationException {
Matcher m;
if ((m = CLASS_WITH_ARGUMENT.matcher(line)).matches()) {
String className = m.group(1);
String arg = m.group(2);
return createFenceMethod(conf, className, arg);
} else if ((m = CLASS_WITHOUT_ARGUMENT.matcher(line)).matches()) {
String className = m.group(1);
return createFenceMethod(conf, className, null);
} else {
throw new BadFencingConfigurationException(
"Unable to parse line: '" + line + "'");
}
}
private static FenceMethodWithArg createFenceMethod(
Configuration conf, String clazzName, String arg)
throws BadFencingConfigurationException {
Class<?> clazz;
try {
// See if it's a short name for one of the built-in methods
clazz = STANDARD_METHODS.get(clazzName);
if (clazz == null) {
// Try to instantiate the user's custom method
clazz = Class.forName(clazzName);
}
} catch (Exception e) {
throw new BadFencingConfigurationException(
"Could not find configured fencing method " + clazzName,
e);
}
// Check that it implements the right interface
if (!FenceMethod.class.isAssignableFrom(clazz)) {
throw new BadFencingConfigurationException("Class " + clazzName +
" does not implement FenceMethod");
}
FenceMethod method = (FenceMethod)ReflectionUtils.newInstance(
clazz, conf);
method.checkArgs(arg);
return new FenceMethodWithArg(method, arg);
}
private static class FenceMethodWithArg {
private final FenceMethod method;
private final String arg;
private FenceMethodWithArg(FenceMethod method, String arg) {
this.method = method;
this.arg = arg;
}
public String toString() {
return method.getClass().getCanonicalName() + "(" + arg + ")";
}
}
}

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.
*/
package org.apache.hadoop.ha;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Exception thrown to indicate that an operation performed
* to modify the state of a service or application failed.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class ServiceFailedException extends IOException {
private static final long serialVersionUID = 1L;
public ServiceFailedException(final String message) {
super(message);
}
public ServiceFailedException(String message, Throwable cause) {
super(message, cause);
}
}

View File

@ -0,0 +1,187 @@
/**
* 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.ha;
import java.io.IOException;
import java.lang.reflect.Field;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.util.StringUtils;
import com.google.common.annotations.VisibleForTesting;
/**
* Fencing method that runs a shell command. It should be specified
* in the fencing configuration like:<br>
* <code>
* shell(/path/to/my/script.sh arg1 arg2 ...)
* </code><br>
* The string between '(' and ')' is passed directly to a bash shell and
* may not include any closing parentheses.<p>
*
* 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.<p>
*
* 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.<p>
*
* <em>Note:</em> 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).
*/
public class ShellCommandFencer
extends Configured implements FenceMethod {
/** Length at which to abbreviate command in long messages */
private static final int ABBREV_LENGTH = 20;
@VisibleForTesting
static Log LOG = LogFactory.getLog(
ShellCommandFencer.class);
@Override
public void checkArgs(String args) throws BadFencingConfigurationException {
if (args == null || args.isEmpty()) {
throw new BadFencingConfigurationException(
"No argument passed to 'shell' fencing method");
}
// Nothing else we can really check without actually running the command
}
@Override
public boolean tryFence(InetSocketAddress serviceAddr, String cmd) {
List<String> cmdList = Arrays.asList(cmd.split("\\s+"));
// Create arg list with service as the first argument
List<String> argList = new ArrayList<String>();
argList.add(cmdList.get(0));
argList.add(serviceAddr.getHostName() + ":" + serviceAddr.getPort());
argList.addAll(cmdList.subList(1, cmdList.size()));
String cmdWithSvc = StringUtils.join(" ", argList);
ProcessBuilder builder = new ProcessBuilder(
"bash", "-e", "-c", cmdWithSvc);
setConfAsEnvVars(builder.environment());
Process p;
try {
p = builder.start();
p.getOutputStream().close();
} catch (IOException e) {
LOG.warn("Unable to execute " + cmd, e);
return false;
}
String pid = tryGetPid(p);
LOG.info("Launched fencing command '" + cmd + "' with "
+ ((pid != null) ? ("pid " + pid) : "unknown pid"));
String logPrefix = abbreviate(cmd, ABBREV_LENGTH);
if (pid != null) {
logPrefix = "[PID " + pid + "] " + logPrefix;
}
// Pump logs to stderr
StreamPumper errPumper = new StreamPumper(
LOG, logPrefix, p.getErrorStream(),
StreamPumper.StreamType.STDERR);
errPumper.start();
StreamPumper outPumper = new StreamPumper(
LOG, logPrefix, p.getInputStream(),
StreamPumper.StreamType.STDOUT);
outPumper.start();
int rc;
try {
rc = p.waitFor();
errPumper.join();
outPumper.join();
} catch (InterruptedException ie) {
LOG.warn("Interrupted while waiting for fencing command: " + cmd);
return false;
}
return rc == 0;
}
/**
* Abbreviate a string by putting '...' in the middle of it,
* in an attempt to keep logs from getting too messy.
* @param cmd the string to abbreviate
* @param len maximum length to abbreviate to
* @return abbreviated string
*/
static String abbreviate(String cmd, int len) {
if (cmd.length() > len && len >= 5) {
int firstHalf = (len - 3) / 2;
int rem = len - firstHalf - 3;
return cmd.substring(0, firstHalf) +
"..." + cmd.substring(cmd.length() - rem);
} else {
return cmd;
}
}
/**
* Attempt to use evil reflection tricks to determine the
* pid of a launched process. This is helpful to ops
* if debugging a fencing process that might have gone
* wrong. If running on a system or JVM where this doesn't
* work, it will simply return null.
*/
private static String tryGetPid(Process p) {
try {
Class<? extends Process> clazz = p.getClass();
if (clazz.getName().equals("java.lang.UNIXProcess")) {
Field f = clazz.getDeclaredField("pid");
f.setAccessible(true);
return String.valueOf(f.getInt(p));
} else {
LOG.trace("Unable to determine pid for " + p
+ " since it is not a UNIXProcess");
return null;
}
} catch (Throwable t) {
LOG.trace("Unable to determine pid for " + p, t);
return null;
}
}
/**
* Set the environment of the subprocess to be the Configuration,
* with '.'s replaced by '_'s.
*/
private void setConfAsEnvVars(Map<String, String> env) {
for (Map.Entry<String, String> pair : getConf()) {
env.put(pair.getKey().replace('.', '_'), pair.getValue());
}
}
}

View File

@ -0,0 +1,315 @@
/**
* 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.ha;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Collection;
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.conf.Configured;
import com.google.common.annotations.VisibleForTesting;
import com.jcraft.jsch.ChannelExec;
import com.jcraft.jsch.JSch;
import com.jcraft.jsch.JSchException;
import com.jcraft.jsch.Session;
/**
* This fencing implementation sshes to the target node and uses
* <code>fuser</code> to kill the process listening on the service's
* TCP port. This is more accurate than using "jps" since it doesn't
* require parsing, and will work even if there are multiple service
* processes running on the same machine.<p>
* It returns a successful status code if:
* <ul>
* <li><code>fuser</code> indicates it successfully killed a process, <em>or</em>
* <li><code>nc -z</code> indicates that nothing is listening on the target port
* </ul>
* <p>
* This fencing mechanism is configured as following in the fencing method
* list:
* <code>sshfence([[username][:ssh-port]])</code>
* where the optional argument specifies the username and port to use
* with ssh.
* <p>
* In order to achieve passwordless SSH, the operator must also configure
* <code>dfs.ha.fencing.ssh.private-key-files<code> to point to an
* SSH key that has passphrase-less access to the given username and host.
*/
public class SshFenceByTcpPort extends Configured
implements FenceMethod {
static final Log LOG = LogFactory.getLog(
SshFenceByTcpPort.class);
static final String CONF_CONNECT_TIMEOUT_KEY =
"dfs.ha.fencing.ssh.connect-timeout";
private static final int CONF_CONNECT_TIMEOUT_DEFAULT =
30*1000;
static final String CONF_IDENTITIES_KEY =
"dfs.ha.fencing.ssh.private-key-files";
/**
* Verify that the argument, if given, in the conf is parseable.
*/
@Override
public void checkArgs(String argStr) throws BadFencingConfigurationException {
if (argStr != null) {
// Use a dummy service when checking the arguments defined
// in the configuration are parseable.
new Args(new InetSocketAddress("localhost", 8020), argStr);
}
}
@Override
public boolean tryFence(InetSocketAddress serviceAddr, String argsStr)
throws BadFencingConfigurationException {
Args args = new Args(serviceAddr, argsStr);
Session session;
try {
session = createSession(args);
} catch (JSchException e) {
LOG.warn("Unable to create SSH session", e);
return false;
}
LOG.info("Connecting to " + args.host + "...");
try {
session.connect(getSshConnectTimeout());
} catch (JSchException e) {
LOG.warn("Unable to connect to " + args.host
+ " as user " + args.user, e);
return false;
}
LOG.info("Connected to " + args.host);
try {
return doFence(session, args.targetPort);
} catch (JSchException e) {
LOG.warn("Unable to achieve fencing on remote host", e);
return false;
} finally {
session.disconnect();
}
}
private Session createSession(Args args) throws JSchException {
JSch jsch = new JSch();
for (String keyFile : getKeyFiles()) {
jsch.addIdentity(keyFile);
}
JSch.setLogger(new LogAdapter());
Session session = jsch.getSession(args.user, args.host, args.sshPort);
session.setConfig("StrictHostKeyChecking", "no");
return session;
}
private boolean doFence(Session session, int port) throws JSchException {
try {
LOG.info("Looking for process running on port " + port);
int rc = execCommand(session,
"PATH=$PATH:/sbin:/usr/sbin fuser -v -k -n tcp " + port);
if (rc == 0) {
LOG.info("Successfully killed process that was " +
"listening on port " + port);
// exit code 0 indicates the process was successfully killed.
return true;
} else if (rc == 1) {
// exit code 1 indicates either that the process was not running
// or that fuser didn't have root privileges in order to find it
// (eg running as a different user)
LOG.info(
"Indeterminate response from trying to kill service. " +
"Verifying whether it is running using nc...");
rc = execCommand(session, "nc -z localhost 8020");
if (rc == 0) {
// the service is still listening - we are unable to fence
LOG.warn("Unable to fence - it is running but we cannot kill it");
return false;
} else {
LOG.info("Verified that the service is down.");
return true;
}
} else {
// other
}
LOG.info("rc: " + rc);
return rc == 0;
} catch (InterruptedException e) {
LOG.warn("Interrupted while trying to fence via ssh", e);
return false;
} catch (IOException e) {
LOG.warn("Unknown failure while trying to fence via ssh", e);
return false;
}
}
/**
* Execute a command through the ssh session, pumping its
* stderr and stdout to our own logs.
*/
private int execCommand(Session session, String cmd)
throws JSchException, InterruptedException, IOException {
LOG.debug("Running cmd: " + cmd);
ChannelExec exec = null;
try {
exec = (ChannelExec)session.openChannel("exec");
exec.setCommand(cmd);
exec.setInputStream(null);
exec.connect();
// Pump stdout of the command to our WARN logs
StreamPumper outPumper = new StreamPumper(LOG, cmd + " via ssh",
exec.getInputStream(), StreamPumper.StreamType.STDOUT);
outPumper.start();
// Pump stderr of the command to our WARN logs
StreamPumper errPumper = new StreamPumper(LOG, cmd + " via ssh",
exec.getErrStream(), StreamPumper.StreamType.STDERR);
errPumper.start();
outPumper.join();
errPumper.join();
return exec.getExitStatus();
} finally {
cleanup(exec);
}
}
private static void cleanup(ChannelExec exec) {
if (exec != null) {
try {
exec.disconnect();
} catch (Throwable t) {
LOG.warn("Couldn't disconnect ssh channel", t);
}
}
}
private int getSshConnectTimeout() {
return getConf().getInt(
CONF_CONNECT_TIMEOUT_KEY, CONF_CONNECT_TIMEOUT_DEFAULT);
}
private Collection<String> getKeyFiles() {
return getConf().getTrimmedStringCollection(CONF_IDENTITIES_KEY);
}
/**
* Container for the parsed arg line for this fencing method.
*/
@VisibleForTesting
static class Args {
private static final Pattern USER_PORT_RE = Pattern.compile(
"([^:]+?)?(?:\\:(\\d+))?");
private static final int DEFAULT_SSH_PORT = 22;
String host;
int targetPort;
String user;
int sshPort;
public Args(InetSocketAddress serviceAddr, String arg)
throws BadFencingConfigurationException {
host = serviceAddr.getHostName();
targetPort = serviceAddr.getPort();
user = System.getProperty("user.name");
sshPort = DEFAULT_SSH_PORT;
// Parse optional user and ssh port
if (arg != null && !"".equals(arg)) {
Matcher m = USER_PORT_RE.matcher(arg);
if (!m.matches()) {
throw new BadFencingConfigurationException(
"Unable to parse user and SSH port: "+ arg);
}
if (m.group(1) != null) {
user = m.group(1);
}
if (m.group(2) != null) {
sshPort = parseConfiggedPort(m.group(2));
}
}
}
private Integer parseConfiggedPort(String portStr)
throws BadFencingConfigurationException {
try {
return Integer.valueOf(portStr);
} catch (NumberFormatException nfe) {
throw new BadFencingConfigurationException(
"Port number '" + portStr + "' invalid");
}
}
}
/**
* Adapter from JSch's logger interface to our log4j
*/
private static class LogAdapter implements com.jcraft.jsch.Logger {
static final Log LOG = LogFactory.getLog(
SshFenceByTcpPort.class.getName() + ".jsch");
public boolean isEnabled(int level) {
switch (level) {
case com.jcraft.jsch.Logger.DEBUG:
return LOG.isDebugEnabled();
case com.jcraft.jsch.Logger.INFO:
return LOG.isInfoEnabled();
case com.jcraft.jsch.Logger.WARN:
return LOG.isWarnEnabled();
case com.jcraft.jsch.Logger.ERROR:
return LOG.isErrorEnabled();
case com.jcraft.jsch.Logger.FATAL:
return LOG.isFatalEnabled();
default:
return false;
}
}
public void log(int level, String message) {
switch (level) {
case com.jcraft.jsch.Logger.DEBUG:
LOG.debug(message);
break;
case com.jcraft.jsch.Logger.INFO:
LOG.info(message);
break;
case com.jcraft.jsch.Logger.WARN:
LOG.warn(message);
break;
case com.jcraft.jsch.Logger.ERROR:
LOG.error(message);
break;
case com.jcraft.jsch.Logger.FATAL:
LOG.fatal(message);
break;
}
}
}
}

View File

@ -0,0 +1,90 @@
/**
* 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.ha;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import org.apache.commons.logging.Log;
/**
* Class responsible for pumping the streams of the subprocess
* out to log4j. stderr is pumped to WARN level and stdout is
* pumped to INFO level
*/
class StreamPumper {
enum StreamType {
STDOUT, STDERR;
}
private final Log log;
final Thread thread;
final String logPrefix;
final StreamPumper.StreamType type;
private final InputStream stream;
private boolean started = false;
StreamPumper(final Log log, final String logPrefix,
final InputStream stream, final StreamType type) {
this.log = log;
this.logPrefix = logPrefix;
this.stream = stream;
this.type = type;
thread = new Thread(new Runnable() {
@Override
public void run() {
try {
pump();
} catch (Throwable t) {
ShellCommandFencer.LOG.warn(logPrefix +
": Unable to pump output from " + type,
t);
}
}
}, logPrefix + ": StreamPumper for " + type);
thread.setDaemon(true);
}
void join() throws InterruptedException {
assert started;
thread.join();
}
void start() {
assert !started;
thread.start();
started = true;
}
protected void pump() throws IOException {
InputStreamReader inputStreamReader = new InputStreamReader(stream);
BufferedReader br = new BufferedReader(inputStreamReader);
String line = null;
while ((line = br.readLine()) != null) {
if (type == StreamType.STDOUT) {
log.info(logPrefix + ": " + line);
} else {
log.warn(logPrefix + ": " + line);
}
}
}
}

View File

@ -0,0 +1,135 @@
/**
* 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.ha.protocolPB;
import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.GetServiceStateRequestProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceStateProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.MonitorHealthRequestProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.ReadyToBecomeActiveRequestProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToActiveRequestProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToStandbyRequestProto;
import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
/**
* This class is the client side translator to translate the requests made on
* {@link HAServiceProtocol} interfaces to the RPC server implementing
* {@link HAServiceProtocolPB}.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class HAServiceProtocolClientSideTranslatorPB implements
HAServiceProtocol, Closeable {
/** RpcController is not used and hence is set to null */
private final static RpcController NULL_CONTROLLER = null;
private final static MonitorHealthRequestProto MONITOR_HEALTH_REQ =
MonitorHealthRequestProto.newBuilder().build();
private final static TransitionToActiveRequestProto TRANSITION_TO_ACTIVE_REQ =
TransitionToActiveRequestProto.newBuilder().build();
private final static TransitionToStandbyRequestProto TRANSITION_TO_STANDBY_REQ =
TransitionToStandbyRequestProto.newBuilder().build();
private final static GetServiceStateRequestProto GET_SERVICE_STATE_REQ =
GetServiceStateRequestProto.newBuilder().build();
private final static ReadyToBecomeActiveRequestProto ACTIVE_READY_REQ =
ReadyToBecomeActiveRequestProto.newBuilder().build();
private final HAServiceProtocolPB rpcProxy;
public HAServiceProtocolClientSideTranslatorPB(InetSocketAddress addr,
Configuration conf) throws IOException {
RPC.setProtocolEngine(conf, HAServiceProtocolPB.class,
ProtobufRpcEngine.class);
rpcProxy = RPC.getProxy(HAServiceProtocolPB.class,
RPC.getProtocolVersion(HAServiceProtocolPB.class), addr, conf);
}
@Override
public void monitorHealth() throws IOException {
try {
rpcProxy.monitorHealth(NULL_CONTROLLER, MONITOR_HEALTH_REQ);
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public void transitionToActive() throws IOException {
try {
rpcProxy.transitionToActive(NULL_CONTROLLER, TRANSITION_TO_ACTIVE_REQ);
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public void transitionToStandby() throws IOException {
try {
rpcProxy.transitionToStandby(NULL_CONTROLLER, TRANSITION_TO_STANDBY_REQ);
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public HAServiceState getServiceState() throws IOException {
HAServiceStateProto state;
try {
state = rpcProxy.getServiceState(NULL_CONTROLLER,
GET_SERVICE_STATE_REQ).getState();
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
switch(state) {
case ACTIVE:
return HAServiceState.ACTIVE;
case STANDBY:
return HAServiceState.STANDBY;
case INITIALIZING:
default:
return HAServiceState.INITIALIZING;
}
}
@Override
public void close() {
RPC.stopProxy(rpcProxy);
}
@Override
public boolean readyToBecomeActive() throws IOException {
try {
return rpcProxy.readyToBecomeActive(NULL_CONTROLLER, ACTIVE_READY_REQ)
.getReadyToBecomeActive();
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
}

View File

@ -0,0 +1,39 @@
/**
* 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.ha.protocolPB;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceProtocolService;
import org.apache.hadoop.ipc.ProtocolInfo;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.KerberosInfo;
@KerberosInfo(
serverPrincipal=CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
@ProtocolInfo(protocolName = "org.apache.hadoop.ha.HAServiceProtocol",
protocolVersion = 1)
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface HAServiceProtocolPB extends
HAServiceProtocolService.BlockingInterface, VersionedProtocol {
/**
* If any methods need annotation, it can be added here
*/
}

View File

@ -0,0 +1,158 @@
/**
* 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.ha.protocolPB;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.GetServiceStateRequestProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.GetServiceStateResponseProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceStateProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.MonitorHealthRequestProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.MonitorHealthResponseProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.ReadyToBecomeActiveRequestProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.ReadyToBecomeActiveResponseProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToActiveRequestProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToActiveResponseProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToStandbyRequestProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToStandbyResponseProto;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
/**
* This class is used on the server side. Calls come across the wire for the
* for protocol {@link HAServiceProtocolPB}.
* This class translates the PB data types
* to the native data types used inside the NN as specified in the generic
* ClientProtocol.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class HAServiceProtocolServerSideTranslatorPB implements
HAServiceProtocolPB {
private final HAServiceProtocol server;
private static final MonitorHealthResponseProto MONITOR_HEALTH_RESP =
MonitorHealthResponseProto.newBuilder().build();
private static final TransitionToActiveResponseProto TRANSITION_TO_ACTIVE_RESP =
TransitionToActiveResponseProto.newBuilder().build();
private static final TransitionToStandbyResponseProto TRANSITION_TO_STANDBY_RESP =
TransitionToStandbyResponseProto.newBuilder().build();
public HAServiceProtocolServerSideTranslatorPB(HAServiceProtocol server) {
this.server = server;
}
@Override
public MonitorHealthResponseProto monitorHealth(RpcController controller,
MonitorHealthRequestProto request) throws ServiceException {
try {
server.monitorHealth();
return MONITOR_HEALTH_RESP;
} catch(IOException e) {
throw new ServiceException(e);
}
}
@Override
public TransitionToActiveResponseProto transitionToActive(
RpcController controller, TransitionToActiveRequestProto request)
throws ServiceException {
try {
server.transitionToActive();
return TRANSITION_TO_ACTIVE_RESP;
} catch(IOException e) {
throw new ServiceException(e);
}
}
@Override
public TransitionToStandbyResponseProto transitionToStandby(
RpcController controller, TransitionToStandbyRequestProto request)
throws ServiceException {
try {
server.transitionToStandby();
return TRANSITION_TO_STANDBY_RESP;
} catch(IOException e) {
throw new ServiceException(e);
}
}
@Override
public GetServiceStateResponseProto getServiceState(RpcController controller,
GetServiceStateRequestProto request) throws ServiceException {
HAServiceState s;
try {
s = server.getServiceState();
} catch(IOException e) {
throw new ServiceException(e);
}
HAServiceStateProto ret;
switch (s) {
case ACTIVE:
ret = HAServiceStateProto.ACTIVE;
break;
case STANDBY:
ret = HAServiceStateProto.STANDBY;
break;
case INITIALIZING:
default:
ret = HAServiceStateProto.INITIALIZING;
break;
}
return GetServiceStateResponseProto.newBuilder().setState(ret).build();
}
@Override
public long getProtocolVersion(String protocol, long clientVersion)
throws IOException {
return RPC.getProtocolVersion(HAServiceProtocolPB.class);
}
@Override
public ProtocolSignature getProtocolSignature(String protocol,
long clientVersion, int clientMethodsHash) throws IOException {
if (!protocol.equals(RPC.getProtocolName(HAServiceProtocolPB.class))) {
throw new IOException("Serverside implements " +
RPC.getProtocolName(HAServiceProtocolPB.class) +
". The following requested protocol is unknown: " + protocol);
}
return ProtocolSignature.getProtocolSignature(clientMethodsHash,
RPC.getProtocolVersion(HAServiceProtocolPB.class),
HAServiceProtocolPB.class);
}
@Override
public ReadyToBecomeActiveResponseProto readyToBecomeActive(
RpcController controller, ReadyToBecomeActiveRequestProto request)
throws ServiceException {
try {
return ReadyToBecomeActiveResponseProto.newBuilder()
.setReadyToBecomeActive(server.readyToBecomeActive()).build();
} catch (IOException e) {
throw new ServiceException(e);
}
}
}

View File

@ -27,28 +27,28 @@ import org.apache.hadoop.ipc.RPC;
* event of failover, and always returns the same proxy object.
*/
@InterfaceStability.Evolving
public class DefaultFailoverProxyProvider implements FailoverProxyProvider {
public class DefaultFailoverProxyProvider<T> implements FailoverProxyProvider<T> {
private Object proxy;
private Class<?> iface;
private T proxy;
private Class<T> iface;
public DefaultFailoverProxyProvider(Class<?> iface, Object proxy) {
public DefaultFailoverProxyProvider(Class<T> iface, T proxy) {
this.proxy = proxy;
this.iface = iface;
}
@Override
public Class<?> getInterface() {
public Class<T> getInterface() {
return iface;
}
@Override
public Object getProxy() {
public T getProxy() {
return proxy;
}
@Override
public void performFailover(Object currentProxy) {
public void performFailover(T currentProxy) {
// Nothing to do.
}

View File

@ -29,7 +29,7 @@ import org.apache.hadoop.classification.InterfaceStability;
* {@link RetryPolicy}.
*/
@InterfaceStability.Evolving
public interface FailoverProxyProvider extends Closeable {
public interface FailoverProxyProvider<T> extends Closeable {
/**
* Get the proxy object which should be used until the next failover event
@ -37,7 +37,7 @@ public interface FailoverProxyProvider extends Closeable {
*
* @return the proxy object to invoke methods upon
*/
public Object getProxy();
public T getProxy();
/**
* Called whenever the associated {@link RetryPolicy} determines that an error
@ -46,7 +46,7 @@ public interface FailoverProxyProvider extends Closeable {
* @param currentProxy the proxy object which was being used before this
* failover event
*/
public void performFailover(Object currentProxy);
public void performFailover(T currentProxy);
/**
* Return a reference to the interface this provider's proxy objects actually
@ -58,5 +58,5 @@ public interface FailoverProxyProvider extends Closeable {
* @return the interface implemented by the proxy objects returned by
* {@link FailoverProxyProvider#getProxy()}
*/
public Class<?> getInterface();
public Class<T> getInterface();
}

View File

@ -20,14 +20,15 @@ package org.apache.hadoop.io.retry;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.lang.reflect.Proxy;
import java.util.Collections;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.retry.RetryPolicy.RetryAction;
import org.apache.hadoop.util.ThreadUtil;
import org.apache.hadoop.ipc.Client.ConnectionId;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcInvocationHandler;
class RetryInvocationHandler implements RpcInvocationHandler {
@ -38,6 +39,7 @@ class RetryInvocationHandler implements RpcInvocationHandler {
* The number of times the associated proxyProvider has ever been failed over.
*/
private long proxyProviderFailoverCount = 0;
private volatile boolean hasMadeASuccessfulCall = false;
private RetryPolicy defaultPolicy;
private Map<String,RetryPolicy> methodNameToPolicyMap;
@ -78,47 +80,82 @@ class RetryInvocationHandler implements RpcInvocationHandler {
invocationAttemptFailoverCount = proxyProviderFailoverCount;
}
try {
return invokeMethod(method, args);
Object ret = invokeMethod(method, args);
hasMadeASuccessfulCall = true;
return ret;
} catch (Exception e) {
boolean isMethodIdempotent = proxyProvider.getInterface()
.getMethod(method.getName(), method.getParameterTypes())
.isAnnotationPresent(Idempotent.class);
RetryAction action = policy.shouldRetry(e, retries++, invocationFailoverCount,
isMethodIdempotent);
if (action == RetryAction.FAIL) {
LOG.warn("Exception while invoking " + method.getName()
+ " of " + currentProxy.getClass() + ". Not retrying.", e);
if (!method.getReturnType().equals(Void.TYPE)) {
throw e; // non-void methods can't fail without an exception
if (action.action == RetryAction.RetryDecision.FAIL) {
if (action.reason != null) {
LOG.warn("Exception while invoking " +
currentProxy.getClass() + "." + method.getName() +
". Not retrying because " + action.reason, e);
}
return null;
} else if (action == RetryAction.FAILOVER_AND_RETRY) {
LOG.warn("Exception while invoking " + method.getName()
+ " of " + currentProxy.getClass()
+ " after " + invocationFailoverCount + " fail over attempts."
+ " Trying to fail over.", e);
// Make sure that concurrent failed method invocations only cause a
// single actual fail over.
synchronized (proxyProvider) {
if (invocationAttemptFailoverCount == proxyProviderFailoverCount) {
proxyProvider.performFailover(currentProxy);
proxyProviderFailoverCount++;
currentProxy = proxyProvider.getProxy();
throw e;
} else { // retry or failover
// avoid logging the failover if this is the first call on this
// proxy object, and we successfully achieve the failover without
// any flip-flopping
boolean worthLogging =
!(invocationFailoverCount == 0 && !hasMadeASuccessfulCall);
worthLogging |= LOG.isDebugEnabled();
if (action.action == RetryAction.RetryDecision.FAILOVER_AND_RETRY &&
worthLogging) {
String msg = "Exception while invoking " + method.getName()
+ " of class " + currentProxy.getClass().getSimpleName();
if (invocationFailoverCount > 0) {
msg += " after " + invocationFailoverCount + " fail over attempts";
}
msg += ". Trying to fail over " + formatSleepMessage(action.delayMillis);
if (LOG.isDebugEnabled()) {
LOG.debug(msg, e);
} else {
LOG.warn("A failover has occurred since the start of this method"
+ " invocation attempt.");
LOG.warn(msg);
}
} else {
if(LOG.isDebugEnabled()) {
LOG.debug("Exception while invoking " + method.getName()
+ " of class " + currentProxy.getClass().getSimpleName() +
". Retrying " + formatSleepMessage(action.delayMillis), e);
}
}
invocationFailoverCount++;
}
if(LOG.isDebugEnabled()) {
LOG.debug("Exception while invoking " + method.getName()
+ " of " + currentProxy.getClass() + ". Retrying.", e);
if (action.delayMillis > 0) {
ThreadUtil.sleepAtLeastIgnoreInterrupts(action.delayMillis);
}
if (action.action == RetryAction.RetryDecision.FAILOVER_AND_RETRY) {
// Make sure that concurrent failed method invocations only cause a
// single actual fail over.
synchronized (proxyProvider) {
if (invocationAttemptFailoverCount == proxyProviderFailoverCount) {
proxyProvider.performFailover(currentProxy);
proxyProviderFailoverCount++;
currentProxy = proxyProvider.getProxy();
} else {
LOG.warn("A failover has occurred since the start of this method"
+ " invocation attempt.");
}
}
invocationFailoverCount++;
}
}
}
}
}
private static String formatSleepMessage(long millis) {
if (millis > 0) {
return "after sleeping for " + millis + "ms.";
} else {
return "immediately.";
}
}
private Object invokeMethod(Method method, Object[] args) throws Throwable {
try {
if (!method.isAccessible()) {
@ -137,9 +174,7 @@ class RetryInvocationHandler implements RpcInvocationHandler {
@Override //RpcInvocationHandler
public ConnectionId getConnectionId() {
RpcInvocationHandler inv = (RpcInvocationHandler) Proxy
.getInvocationHandler(currentProxy);
return inv.getConnectionId();
return RPC.getConnectionIdForProxy(currentProxy);
}
}

View File

@ -33,6 +33,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.StandbyException;
import com.google.common.annotations.VisibleForTesting;
/**
* <p>
* A collection of useful implementations of {@link RetryPolicy}.
@ -42,6 +44,8 @@ public class RetryPolicies {
public static final Log LOG = LogFactory.getLog(RetryPolicies.class);
private static final Random RAND = new Random();
/**
* <p>
* Try once, and fail by re-throwing the exception.
@ -50,14 +54,6 @@ public class RetryPolicies {
*/
public static final RetryPolicy TRY_ONCE_THEN_FAIL = new TryOnceThenFail();
/**
* <p>
* Try once, and fail silently for <code>void</code> methods, or by
* re-throwing the exception for non-<code>void</code> methods.
* </p>
*/
public static final RetryPolicy TRY_ONCE_DONT_FAIL = new TryOnceDontFail();
/**
* <p>
* Keep trying forever.
@ -137,16 +133,17 @@ public class RetryPolicies {
public static final RetryPolicy failoverOnNetworkException(
RetryPolicy fallbackPolicy, int maxFailovers) {
return new FailoverOnNetworkExceptionRetry(fallbackPolicy, maxFailovers);
return failoverOnNetworkException(fallbackPolicy, maxFailovers, 0, 0);
}
public static final RetryPolicy failoverOnNetworkException(
RetryPolicy fallbackPolicy, int maxFailovers, long delayMillis,
long maxDelayBase) {
return new FailoverOnNetworkExceptionRetry(fallbackPolicy, maxFailovers,
delayMillis, maxDelayBase);
}
static class TryOnceThenFail implements RetryPolicy {
public RetryAction shouldRetry(Exception e, int retries, int failovers,
boolean isMethodIdempotent) throws Exception {
throw e;
}
}
static class TryOnceDontFail implements RetryPolicy {
public RetryAction shouldRetry(Exception e, int retries, int failovers,
boolean isMethodIdempotent) throws Exception {
return RetryAction.FAIL;
@ -174,14 +171,10 @@ public class RetryPolicies {
public RetryAction shouldRetry(Exception e, int retries, int failovers,
boolean isMethodIdempotent) throws Exception {
if (retries >= maxRetries) {
throw e;
return RetryAction.FAIL;
}
try {
timeUnit.sleep(calculateSleepTime(retries));
} catch (InterruptedException ie) {
// retry
}
return RetryAction.RETRY;
return new RetryAction(RetryAction.RetryDecision.RETRY,
timeUnit.toMillis(calculateSleepTime(retries)));
}
protected abstract long calculateSleepTime(int retries);
@ -268,7 +261,7 @@ public class RetryPolicies {
}
static class ExponentialBackoffRetry extends RetryLimited {
private Random r = new Random();
public ExponentialBackoffRetry(
int maxRetries, long sleepTime, TimeUnit timeUnit) {
super(maxRetries, sleepTime, timeUnit);
@ -276,16 +269,19 @@ public class RetryPolicies {
@Override
protected long calculateSleepTime(int retries) {
return sleepTime*r.nextInt(1<<(retries+1));
return calculateExponentialTime(sleepTime, retries + 1);
}
}
/*
/**
* Fail over and retry in the case of:
* Remote StandbyException (server is up, but is not the active server)
* Immediate socket exceptions (e.g. no route to host, econnrefused)
* Socket exceptions after initial connection when operation is idempotent
*
* The first failover is immediate, while all subsequent failovers wait an
* exponentially-increasing random amount of time.
*
* Fail immediately in the case of:
* Socket exceptions after initial connection when operation is not idempotent
*
@ -295,33 +291,49 @@ public class RetryPolicies {
private RetryPolicy fallbackPolicy;
private int maxFailovers;
private long delayMillis;
private long maxDelayBase;
public FailoverOnNetworkExceptionRetry(RetryPolicy fallbackPolicy,
int maxFailovers) {
this(fallbackPolicy, maxFailovers, 0, 0);
}
public FailoverOnNetworkExceptionRetry(RetryPolicy fallbackPolicy,
int maxFailovers, long delayMillis, long maxDelayBase) {
this.fallbackPolicy = fallbackPolicy;
this.maxFailovers = maxFailovers;
this.delayMillis = delayMillis;
this.maxDelayBase = maxDelayBase;
}
@Override
public RetryAction shouldRetry(Exception e, int retries,
int failovers, boolean isMethodIdempotent) throws Exception {
if (failovers >= maxFailovers) {
LOG.info("Failovers (" + failovers + ") exceeded maximum allowed ("
return new RetryAction(RetryAction.RetryDecision.FAIL, 0,
"failovers (" + failovers + ") exceeded maximum allowed ("
+ maxFailovers + ")");
return RetryAction.FAIL;
}
if (e instanceof ConnectException ||
e instanceof NoRouteToHostException ||
e instanceof UnknownHostException ||
e instanceof StandbyException) {
return RetryAction.FAILOVER_AND_RETRY;
e instanceof StandbyException ||
isWrappedStandbyException(e)) {
return new RetryAction(
RetryAction.RetryDecision.FAILOVER_AND_RETRY,
// retry immediately if this is our first failover, sleep otherwise
failovers == 0 ? 0 :
calculateExponentialTime(delayMillis, failovers, maxDelayBase));
} else if (e instanceof SocketException ||
e instanceof IOException) {
(e instanceof IOException && !(e instanceof RemoteException))) {
if (isMethodIdempotent) {
return RetryAction.FAILOVER_AND_RETRY;
} else {
return RetryAction.FAIL;
return new RetryAction(RetryAction.RetryDecision.FAIL, 0,
"the invoked method is not idempotent, and unable to determine " +
"whether it was invoked");
}
} else {
return fallbackPolicy.shouldRetry(e, retries, failovers,
@ -330,4 +342,34 @@ public class RetryPolicies {
}
}
/**
* Return a value which is <code>time</code> increasing exponentially as a
* function of <code>retries</code>, +/- 0%-50% of that value, chosen
* randomly.
*
* @param time the base amount of time to work with
* @param retries the number of retries that have so occurred so far
* @param cap value at which to cap the base sleep time
* @return an amount of time to sleep
*/
@VisibleForTesting
public static long calculateExponentialTime(long time, int retries,
long cap) {
long baseTime = Math.min(time * ((long)1 << retries), cap);
return (long) (baseTime * (RAND.nextFloat() + 0.5));
}
private static long calculateExponentialTime(long time, int retries) {
return calculateExponentialTime(time, retries, Long.MAX_VALUE);
}
private static boolean isWrappedStandbyException(Exception e) {
if (!(e instanceof RemoteException)) {
return false;
}
Exception unwrapped = ((RemoteException)e).unwrapRemoteException(
StandbyException.class);
return unwrapped instanceof StandbyException;
}
}

View File

@ -19,7 +19,6 @@ package org.apache.hadoop.io.retry;
import org.apache.hadoop.classification.InterfaceStability;
/**
* <p>
* Specifies a policy for retrying method failures.
@ -33,10 +32,39 @@ public interface RetryPolicy {
* Returned by {@link RetryPolicy#shouldRetry(Exception, int, int, boolean)}.
*/
@InterfaceStability.Evolving
public enum RetryAction {
FAIL,
RETRY,
FAILOVER_AND_RETRY
public static class RetryAction {
// A few common retry policies, with no delays.
public static final RetryAction FAIL =
new RetryAction(RetryDecision.FAIL);
public static final RetryAction RETRY =
new RetryAction(RetryDecision.RETRY);
public static final RetryAction FAILOVER_AND_RETRY =
new RetryAction(RetryDecision.FAILOVER_AND_RETRY);
public final RetryDecision action;
public final long delayMillis;
public final String reason;
public RetryAction(RetryDecision action) {
this(action, 0, null);
}
public RetryAction(RetryDecision action, long delayTime) {
this(action, delayTime, null);
}
public RetryAction(RetryDecision action, long delayTime, String reason) {
this.action = action;
this.delayMillis = delayTime;
this.reason = reason;
}
public enum RetryDecision {
FAIL,
RETRY,
FAILOVER_AND_RETRY
}
}
/**

View File

@ -227,6 +227,8 @@ public class Client {
private int maxIdleTime; //connections will be culled if it was idle for
//maxIdleTime msecs
private int maxRetries; //the max. no. of retries for socket connections
// the max. no. of retries for socket connections on time out exceptions
private int maxRetriesOnSocketTimeouts;
private boolean tcpNoDelay; // if T then disable Nagle's Algorithm
private boolean doPing; //do we need to send ping message
private int pingInterval; // how often sends ping to the server in msecs
@ -250,6 +252,7 @@ public class Client {
this.rpcTimeout = remoteId.getRpcTimeout();
this.maxIdleTime = remoteId.getMaxIdleTime();
this.maxRetries = remoteId.getMaxRetries();
this.maxRetriesOnSocketTimeouts = remoteId.getMaxRetriesOnSocketTimeouts();
this.tcpNoDelay = remoteId.getTcpNoDelay();
this.doPing = remoteId.getDoPing();
this.pingInterval = remoteId.getPingInterval();
@ -478,11 +481,8 @@ public class Client {
if (updateAddress()) {
timeoutFailures = ioFailures = 0;
}
/*
* The max number of retries is 45, which amounts to 20s*45 = 15
* minutes retries.
*/
handleConnectionFailure(timeoutFailures++, 45, toe);
handleConnectionFailure(timeoutFailures++,
maxRetriesOnSocketTimeouts, toe);
} catch (IOException ie) {
if (updateAddress()) {
timeoutFailures = ioFailures = 0;
@ -1286,6 +1286,8 @@ public class Client {
private final int maxIdleTime; //connections will be culled if it was idle for
//maxIdleTime msecs
private final int maxRetries; //the max. no. of retries for socket connections
// the max. no. of retries for socket connections on time out exceptions
private final int maxRetriesOnSocketTimeouts;
private final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
private final boolean doPing; //do we need to send ping message
private final int pingInterval; // how often sends ping to the server in msecs
@ -1293,8 +1295,8 @@ public class Client {
ConnectionId(InetSocketAddress address, Class<?> protocol,
UserGroupInformation ticket, int rpcTimeout,
String serverPrincipal, int maxIdleTime,
int maxRetries, boolean tcpNoDelay,
boolean doPing, int pingInterval) {
int maxRetries, int maxRetriesOnSocketTimeouts,
boolean tcpNoDelay, boolean doPing, int pingInterval) {
this.protocol = protocol;
this.address = address;
this.ticket = ticket;
@ -1302,6 +1304,7 @@ public class Client {
this.serverPrincipal = serverPrincipal;
this.maxIdleTime = maxIdleTime;
this.maxRetries = maxRetries;
this.maxRetriesOnSocketTimeouts = maxRetriesOnSocketTimeouts;
this.tcpNoDelay = tcpNoDelay;
this.doPing = doPing;
this.pingInterval = pingInterval;
@ -1335,6 +1338,11 @@ public class Client {
return maxRetries;
}
/** max connection retries on socket time outs */
public int getMaxRetriesOnSocketTimeouts() {
return maxRetriesOnSocketTimeouts;
}
boolean getTcpNoDelay() {
return tcpNoDelay;
}
@ -1369,6 +1377,9 @@ public class Client {
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_DEFAULT),
conf.getInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_DEFAULT),
conf.getInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT),
conf.getBoolean(CommonConfigurationKeysPublic.IPC_CLIENT_TCPNODELAY_KEY,
CommonConfigurationKeysPublic.IPC_CLIENT_TCPNODELAY_DEFAULT),
doPing,

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.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
/**
* An interface implemented by client-side protocol translators to get the
* underlying proxy object the translator is operating on.
*/
@InterfaceAudience.Private
public interface ProtocolTranslator {
/**
* Return the proxy object underlying this protocol translator.
* @return the proxy object underlying this protocol translator.
*/
public Object getUnderlyingProxyObject();
}

View File

@ -40,6 +40,7 @@ import javax.net.SocketFactory;
import org.apache.commons.logging.*;
import org.apache.hadoop.io.*;
import org.apache.hadoop.ipc.Client.ConnectionId;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolInfoService;
import org.apache.hadoop.net.NetUtils;
@ -530,9 +531,24 @@ public class RPC {
* Returns the server address for a given proxy.
*/
public static InetSocketAddress getServerAddress(Object proxy) {
return getConnectionIdForProxy(proxy).getAddress();
}
/**
* Return the connection ID of the given object. If the provided object is in
* fact a protocol translator, we'll get the connection ID of the underlying
* proxy object.
*
* @param proxy the proxy object to get the connection ID of.
* @return the connection ID for the provided proxy object.
*/
public static ConnectionId getConnectionIdForProxy(Object proxy) {
if (proxy instanceof ProtocolTranslator) {
proxy = ((ProtocolTranslator)proxy).getUnderlyingProxyObject();
}
RpcInvocationHandler inv = (RpcInvocationHandler) Proxy
.getInvocationHandler(proxy);
return inv.getConnectionId().getAddress();
return inv.getConnectionId();
}
/**
@ -564,6 +580,12 @@ public class RPC {
* @param proxy the RPC proxy object to be stopped
*/
public static void stopProxy(Object proxy) {
if (proxy instanceof ProtocolTranslator) {
RPC.stopProxy(((ProtocolTranslator)proxy)
.getUnderlyingProxyObject());
return;
}
InvocationHandler invocationHandler = null;
try {
invocationHandler = Proxy.getInvocationHandler(proxy);

View File

@ -1671,6 +1671,10 @@ public abstract class Server {
// on the server side, as opposed to just a normal exceptional
// result.
LOG.warn(logMsg, e);
} else if (e instanceof StandbyException) {
// Don't log the whole stack trace of these exceptions.
// Way too noisy!
LOG.info(logMsg);
} else {
LOG.info(logMsg, e);
}

View File

@ -17,6 +17,8 @@
*/
package org.apache.hadoop.ipc;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceStability;
/**
@ -24,7 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability;
* set of servers in which only a subset may be active.
*/
@InterfaceStability.Evolving
public class StandbyException extends Exception {
public class StandbyException extends IOException {
static final long serialVersionUID = 0x12308AD010L;
public StandbyException(String msg) {
super(msg);

View File

@ -23,6 +23,7 @@ import java.net.URI;
import java.net.URL;
import java.net.UnknownHostException;
import java.security.AccessController;
import java.security.PrivilegedAction;
import java.util.Arrays;
import java.util.List;
import java.util.ServiceLoader;
@ -448,6 +449,27 @@ public class SecurityUtil {
return buildTokenService(NetUtils.createSocketAddr(uri.getAuthority()));
}
/**
* Perform the given action as the daemon's login user. If the login
* user cannot be determined, this will log a FATAL error and exit
* the whole JVM.
*/
public static <T> T doAsLoginUserOrFatal(PrivilegedAction<T> action) {
if (UserGroupInformation.isSecurityEnabled()) {
UserGroupInformation ugi = null;
try {
ugi = UserGroupInformation.getLoginUser();
} catch (IOException e) {
LOG.fatal("Exception while getting login user", e);
e.printStackTrace();
Runtime.getRuntime().exit(-1);
}
return ugi.doAs(action);
} else {
return action.run();
}
}
/**
* Resolves a host subject to the security requirements determined by
* hadoop.security.token.service.use_ip.
@ -597,5 +619,5 @@ public class SecurityUtil {
void setSearchDomains(String ... domains) {
searchDomains = Arrays.asList(domains);
}
}
}
}

View File

@ -40,6 +40,8 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.util.Daemon;
import com.google.common.base.Preconditions;
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceStability.Evolving
public abstract
@ -84,6 +86,12 @@ extends AbstractDelegationTokenIdentifier>
private Thread tokenRemoverThread;
protected volatile boolean running;
/**
* If the delegation token update thread holds this lock, it will
* not get interrupted.
*/
protected Object noInterruptsLock = new Object();
public AbstractDelegationTokenSecretManager(long delegationKeyUpdateInterval,
long delegationTokenMaxLifetime, long delegationTokenRenewInterval,
long delegationTokenRemoverScanInterval) {
@ -95,6 +103,7 @@ extends AbstractDelegationTokenIdentifier>
/** should be called before this object is used */
public void startThreads() throws IOException {
Preconditions.checkState(!running);
updateCurrentKey();
synchronized (this) {
running = true;
@ -354,12 +363,21 @@ extends AbstractDelegationTokenIdentifier>
}
}
public synchronized void stopThreads() {
public void stopThreads() {
if (LOG.isDebugEnabled())
LOG.debug("Stopping expired delegation token remover thread");
running = false;
if (tokenRemoverThread != null) {
tokenRemoverThread.interrupt();
synchronized (noInterruptsLock) {
tokenRemoverThread.interrupt();
}
try {
tokenRemoverThread.join();
} catch (InterruptedException e) {
throw new RuntimeException(
"Unable to join on token removal thread", e);
}
}
}
@ -395,7 +413,7 @@ extends AbstractDelegationTokenIdentifier>
lastTokenCacheCleanup = now;
}
try {
Thread.sleep(5000); // 5 seconds
Thread.sleep(Math.min(5000, keyUpdateInterval)); // 5 seconds
} catch (InterruptedException ie) {
LOG
.error("InterruptedExcpetion recieved for ExpiredTokenRemover thread "

View File

@ -0,0 +1,49 @@
/*
* 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.util;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceStability;
@InterfaceStability.Evolving
public class ThreadUtil {
private static final Log LOG = LogFactory.getLog(ThreadUtil.class);
/**
* Cause the current thread to sleep as close as possible to the provided
* number of milliseconds. This method will log and ignore any
* {@link InterruptedException} encountered.
*
* @param millis the number of milliseconds for the current thread to sleep
*/
public static void sleepAtLeastIgnoreInterrupts(long millis) {
long start = System.currentTimeMillis();
while (System.currentTimeMillis() - start < millis) {
long timeToSleep = millis -
(System.currentTimeMillis() - start);
try {
Thread.sleep(timeToSleep);
} catch (InterruptedException ie) {
LOG.warn("interrupted while sleeping", ie);
}
}
}
}

View File

@ -216,6 +216,13 @@
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.ha.service.protocol.acl</name>
<value>*</value>
<description>ACL for HAService protocol used by HAAdmin to manage the
active and stand-by states of namenode.</description>
</property>
<property>
<name>security.mrhs.client.protocol.acl</name>

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.
*/
option java_package = "org.apache.hadoop.ha.proto";
option java_outer_classname = "HAServiceProtocolProtos";
option java_generic_services = true;
option java_generate_equals_and_hash = true;
enum HAServiceStateProto {
INITIALIZING = 0;
ACTIVE = 1;
STANDBY = 2;
}
/**
* void request
*/
message MonitorHealthRequestProto {
}
/**
* void response
*/
message MonitorHealthResponseProto {
}
/**
* void request
*/
message TransitionToActiveRequestProto {
}
/**
* void response
*/
message TransitionToActiveResponseProto {
}
/**
* void request
*/
message TransitionToStandbyRequestProto {
}
/**
* void response
*/
message TransitionToStandbyResponseProto {
}
/**
* void request
*/
message GetServiceStateRequestProto {
}
/**
* Returns the state of the service
*/
message GetServiceStateResponseProto {
required HAServiceStateProto state = 1;
}
/**
* void request
*/
message ReadyToBecomeActiveRequestProto {
}
/**
* Returns true if service is ready to become active
*/
message ReadyToBecomeActiveResponseProto {
required bool readyToBecomeActive = 1;
}
/**
* Protocol interface provides High availability related
* primitives to monitor and failover a service.
*
* For details see o.a.h.ha.HAServiceProtocol.
*/
service HAServiceProtocolService {
/**
* Monitor the health of a service.
*/
rpc monitorHealth(MonitorHealthRequestProto)
returns(MonitorHealthResponseProto);
/**
* Request service to tranisition to active state.
*/
rpc transitionToActive(TransitionToActiveRequestProto)
returns(TransitionToActiveResponseProto);
/**
* Request service to transition to standby state.
*/
rpc transitionToStandby(TransitionToStandbyRequestProto)
returns(TransitionToStandbyResponseProto);
/**
* Get the current state of the service.
*/
rpc getServiceState(GetServiceStateRequestProto)
returns(GetServiceStateResponseProto);
/**
* Check if the service is ready to become active
*/
rpc readyToBecomeActive(ReadyToBecomeActiveRequestProto)
returns(ReadyToBecomeActiveResponseProto);
}

View File

@ -487,6 +487,14 @@
</description>
</property>
<property>
<name>ipc.client.connect.max.retries.on.timeouts</name>
<value>45</value>
<description>Indicates the number of retries a client will make on socket timeout
to establish a server connection.
</description>
</property>
<property>
<name>ipc.server.listen.queue.size</name>
<value>128</value>
@ -849,4 +857,30 @@
</description>
</property>
<property>
<name>dfs.ha.fencing.methods</name>
<value></value>
<description>
List of fencing methods to use for service fencing. May contain
builtin methods (eg shell and sshfence) or user-defined method.
</description>
</property>
<property>
<name>dfs.ha.fencing.ssh.connect-timeout</name>
<value>30000</value>
<description>
SSH connection timeout, in milliseconds, to use with the builtin
sshfence fencer.
</description>
</property>
<property>
<name>dfs.ha.fencing.ssh.private-key-files</name>
<value></value>
<description>
The SSH private key files to use with the builtin sshfence fencer.
</description>
</property>
</configuration>

View File

@ -0,0 +1,527 @@
/**
* 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.ha;
import java.io.IOException;
import java.util.List;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.Watcher.Event;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.ZooDefs.Ids;
import org.junit.Before;
import org.junit.Test;
import org.junit.Assert;
import org.mockito.Mockito;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.ha.ActiveStandbyElector.ActiveStandbyElectorCallback;
import org.apache.hadoop.ha.ActiveStandbyElector.ActiveNotFoundException;
public class TestActiveStandbyElector {
static ZooKeeper mockZK;
static int count;
static ActiveStandbyElectorCallback mockApp;
static final byte[] data = new byte[8];
ActiveStandbyElectorTester elector;
class ActiveStandbyElectorTester extends ActiveStandbyElector {
ActiveStandbyElectorTester(String hostPort, int timeout, String parent,
List<ACL> acl, ActiveStandbyElectorCallback app) throws IOException {
super(hostPort, timeout, parent, acl, app);
}
@Override
public ZooKeeper getNewZooKeeper() {
++TestActiveStandbyElector.count;
return TestActiveStandbyElector.mockZK;
}
}
private static final String zkParentName = "/zookeeper";
private static final String zkLockPathName = "/zookeeper/"
+ ActiveStandbyElector.LOCKFILENAME;
@Before
public void init() throws IOException {
count = 0;
mockZK = Mockito.mock(ZooKeeper.class);
mockApp = Mockito.mock(ActiveStandbyElectorCallback.class);
elector = new ActiveStandbyElectorTester("hostPort", 1000, zkParentName,
Ids.OPEN_ACL_UNSAFE, mockApp);
}
/**
* verify that joinElection checks for null data
*/
@Test(expected = HadoopIllegalArgumentException.class)
public void testJoinElectionException() {
elector.joinElection(null);
}
/**
* verify that joinElection tries to create ephemeral lock znode
*/
@Test
public void testJoinElection() {
elector.joinElection(data);
Mockito.verify(mockZK, Mockito.times(1)).create(zkLockPathName, data,
Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
}
/**
* verify that successful znode create result becomes active and monitoring is
* started
*/
@Test
public void testCreateNodeResultBecomeActive() {
elector.joinElection(data);
elector.processResult(Code.OK.intValue(), zkLockPathName, null,
zkLockPathName);
Mockito.verify(mockApp, Mockito.times(1)).becomeActive();
Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
elector, null);
// monitor callback verifies the leader is ephemeral owner of lock but does
// not call becomeActive since its already active
Stat stat = new Stat();
stat.setEphemeralOwner(1L);
Mockito.when(mockZK.getSessionId()).thenReturn(1L);
elector.processResult(Code.OK.intValue(), zkLockPathName, null, stat);
// should not call neutral mode/standby/active
Mockito.verify(mockApp, Mockito.times(0)).enterNeutralMode();
Mockito.verify(mockApp, Mockito.times(0)).becomeStandby();
Mockito.verify(mockApp, Mockito.times(1)).becomeActive();
// another joinElection not called.
Mockito.verify(mockZK, Mockito.times(1)).create(zkLockPathName, data,
Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
// no new monitor called
Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
elector, null);
}
/**
* verify that znode create for existing node and no retry becomes standby and
* monitoring is started
*/
@Test
public void testCreateNodeResultBecomeStandby() {
elector.joinElection(data);
elector.processResult(Code.NODEEXISTS.intValue(), zkLockPathName, null,
zkLockPathName);
Mockito.verify(mockApp, Mockito.times(1)).becomeStandby();
Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
elector, null);
}
/**
* verify that znode create error result in fatal error
*/
@Test
public void testCreateNodeResultError() {
elector.joinElection(data);
elector.processResult(Code.APIERROR.intValue(), zkLockPathName, null,
zkLockPathName);
Mockito.verify(mockApp, Mockito.times(1)).notifyFatalError(
"Received create error from Zookeeper. code:APIERROR");
}
/**
* verify that retry of network errors verifies master by session id and
* becomes active if they match. monitoring is started.
*/
@Test
public void testCreateNodeResultRetryBecomeActive() {
elector.joinElection(data);
elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
zkLockPathName);
elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
zkLockPathName);
elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
zkLockPathName);
elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
zkLockPathName);
// 4 errors results in fatalError
Mockito
.verify(mockApp, Mockito.times(1))
.notifyFatalError(
"Received create error from Zookeeper. code:CONNECTIONLOSS. "+
"Not retrying further znode create connection errors.");
elector.joinElection(data);
// recreate connection via getNewZooKeeper
Assert.assertEquals(2, TestActiveStandbyElector.count);
elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
zkLockPathName);
elector.processResult(Code.NODEEXISTS.intValue(), zkLockPathName, null,
zkLockPathName);
Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
elector, null);
Stat stat = new Stat();
stat.setEphemeralOwner(1L);
Mockito.when(mockZK.getSessionId()).thenReturn(1L);
elector.processResult(Code.OK.intValue(), zkLockPathName, null, stat);
Mockito.verify(mockApp, Mockito.times(1)).becomeActive();
Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
elector, null);
Mockito.verify(mockZK, Mockito.times(6)).create(zkLockPathName, data,
Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
}
/**
* verify that retry of network errors verifies active by session id and
* becomes standby if they dont match. monitoring is started.
*/
@Test
public void testCreateNodeResultRetryBecomeStandby() {
elector.joinElection(data);
elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
zkLockPathName);
elector.processResult(Code.NODEEXISTS.intValue(), zkLockPathName, null,
zkLockPathName);
Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
elector, null);
Stat stat = new Stat();
stat.setEphemeralOwner(0);
Mockito.when(mockZK.getSessionId()).thenReturn(1L);
elector.processResult(Code.OK.intValue(), zkLockPathName, null, stat);
Mockito.verify(mockApp, Mockito.times(1)).becomeStandby();
Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
elector, null);
}
/**
* verify that if create znode results in nodeexists and that znode is deleted
* before exists() watch is set then the return of the exists() method results
* in attempt to re-create the znode and become active
*/
@Test
public void testCreateNodeResultRetryNoNode() {
elector.joinElection(data);
elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
zkLockPathName);
elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
zkLockPathName);
elector.processResult(Code.NODEEXISTS.intValue(), zkLockPathName, null,
zkLockPathName);
Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
elector, null);
elector.processResult(Code.NONODE.intValue(), zkLockPathName, null,
(Stat) null);
Mockito.verify(mockApp, Mockito.times(1)).enterNeutralMode();
Mockito.verify(mockZK, Mockito.times(4)).create(zkLockPathName, data,
Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
}
/**
* verify that more than 3 network error retries result fatalError
*/
@Test
public void testStatNodeRetry() {
elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
(Stat) null);
elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
(Stat) null);
elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
(Stat) null);
elector.processResult(Code.CONNECTIONLOSS.intValue(), zkLockPathName, null,
(Stat) null);
Mockito
.verify(mockApp, Mockito.times(1))
.notifyFatalError(
"Received stat error from Zookeeper. code:CONNECTIONLOSS. "+
"Not retrying further znode monitoring connection errors.");
}
/**
* verify error in exists() callback results in fatal error
*/
@Test
public void testStatNodeError() {
elector.processResult(Code.RUNTIMEINCONSISTENCY.intValue(), zkLockPathName,
null, (Stat) null);
Mockito.verify(mockApp, Mockito.times(0)).enterNeutralMode();
Mockito.verify(mockApp, Mockito.times(1)).notifyFatalError(
"Received stat error from Zookeeper. code:RUNTIMEINCONSISTENCY");
}
/**
* verify behavior of watcher.process callback with non-node event
*/
@Test
public void testProcessCallbackEventNone() {
elector.joinElection(data);
WatchedEvent mockEvent = Mockito.mock(WatchedEvent.class);
Mockito.when(mockEvent.getType()).thenReturn(Event.EventType.None);
// first SyncConnected should not do anything
Mockito.when(mockEvent.getState()).thenReturn(
Event.KeeperState.SyncConnected);
elector.process(mockEvent);
Mockito.verify(mockZK, Mockito.times(0)).exists(Mockito.anyString(),
Mockito.anyBoolean(), Mockito.<AsyncCallback.StatCallback> anyObject(),
Mockito.<Object> anyObject());
// disconnection should enter safe mode
Mockito.when(mockEvent.getState()).thenReturn(
Event.KeeperState.Disconnected);
elector.process(mockEvent);
Mockito.verify(mockApp, Mockito.times(1)).enterNeutralMode();
// re-connection should monitor master status
Mockito.when(mockEvent.getState()).thenReturn(
Event.KeeperState.SyncConnected);
elector.process(mockEvent);
Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
elector, null);
// session expired should enter safe mode and initiate re-election
// re-election checked via checking re-creation of new zookeeper and
// call to create lock znode
Mockito.when(mockEvent.getState()).thenReturn(Event.KeeperState.Expired);
elector.process(mockEvent);
// already in safe mode above. should not enter safe mode again
Mockito.verify(mockApp, Mockito.times(1)).enterNeutralMode();
// called getNewZooKeeper to create new session. first call was in
// constructor
Assert.assertEquals(2, TestActiveStandbyElector.count);
// once in initial joinElection and one now
Mockito.verify(mockZK, Mockito.times(2)).create(zkLockPathName, data,
Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
// create znode success. become master and monitor
elector.processResult(Code.OK.intValue(), zkLockPathName, null,
zkLockPathName);
Mockito.verify(mockApp, Mockito.times(1)).becomeActive();
Mockito.verify(mockZK, Mockito.times(2)).exists(zkLockPathName, true,
elector, null);
// error event results in fatal error
Mockito.when(mockEvent.getState()).thenReturn(Event.KeeperState.AuthFailed);
elector.process(mockEvent);
Mockito.verify(mockApp, Mockito.times(1)).notifyFatalError(
"Unexpected Zookeeper watch event state: AuthFailed");
// only 1 state change callback is called at a time
Mockito.verify(mockApp, Mockito.times(1)).enterNeutralMode();
}
/**
* verify behavior of watcher.process with node event
*/
@Test
public void testProcessCallbackEventNode() {
elector.joinElection(data);
// make the object go into the monitoring state
elector.processResult(Code.NODEEXISTS.intValue(), zkLockPathName, null,
zkLockPathName);
Mockito.verify(mockApp, Mockito.times(1)).becomeStandby();
Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
elector, null);
WatchedEvent mockEvent = Mockito.mock(WatchedEvent.class);
Mockito.when(mockEvent.getPath()).thenReturn(zkLockPathName);
// monitoring should be setup again after event is received
Mockito.when(mockEvent.getType()).thenReturn(
Event.EventType.NodeDataChanged);
elector.process(mockEvent);
Mockito.verify(mockZK, Mockito.times(2)).exists(zkLockPathName, true,
elector, null);
// monitoring should be setup again after event is received
Mockito.when(mockEvent.getType()).thenReturn(
Event.EventType.NodeChildrenChanged);
elector.process(mockEvent);
Mockito.verify(mockZK, Mockito.times(3)).exists(zkLockPathName, true,
elector, null);
// lock node deletion when in standby mode should create znode again
// successful znode creation enters active state and sets monitor
Mockito.when(mockEvent.getType()).thenReturn(Event.EventType.NodeDeleted);
elector.process(mockEvent);
// enterNeutralMode not called when app is standby and leader is lost
Mockito.verify(mockApp, Mockito.times(0)).enterNeutralMode();
// once in initial joinElection() and one now
Mockito.verify(mockZK, Mockito.times(2)).create(zkLockPathName, data,
Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
elector.processResult(Code.OK.intValue(), zkLockPathName, null,
zkLockPathName);
Mockito.verify(mockApp, Mockito.times(1)).becomeActive();
Mockito.verify(mockZK, Mockito.times(4)).exists(zkLockPathName, true,
elector, null);
// lock node deletion in active mode should enter neutral mode and create
// znode again successful znode creation enters active state and sets
// monitor
Mockito.when(mockEvent.getType()).thenReturn(Event.EventType.NodeDeleted);
elector.process(mockEvent);
Mockito.verify(mockApp, Mockito.times(1)).enterNeutralMode();
// another joinElection called
Mockito.verify(mockZK, Mockito.times(3)).create(zkLockPathName, data,
Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
elector.processResult(Code.OK.intValue(), zkLockPathName, null,
zkLockPathName);
Mockito.verify(mockApp, Mockito.times(2)).becomeActive();
Mockito.verify(mockZK, Mockito.times(5)).exists(zkLockPathName, true,
elector, null);
// bad path name results in fatal error
Mockito.when(mockEvent.getPath()).thenReturn(null);
elector.process(mockEvent);
Mockito.verify(mockApp, Mockito.times(1)).notifyFatalError(
"Unexpected watch error from Zookeeper");
// fatal error means no new connection other than one from constructor
Assert.assertEquals(1, TestActiveStandbyElector.count);
// no new watches after fatal error
Mockito.verify(mockZK, Mockito.times(5)).exists(zkLockPathName, true,
elector, null);
}
/**
* verify becomeStandby is not called if already in standby
*/
@Test
public void testSuccessiveStandbyCalls() {
elector.joinElection(data);
// make the object go into the monitoring standby state
elector.processResult(Code.NODEEXISTS.intValue(), zkLockPathName, null,
zkLockPathName);
Mockito.verify(mockApp, Mockito.times(1)).becomeStandby();
Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
elector, null);
WatchedEvent mockEvent = Mockito.mock(WatchedEvent.class);
Mockito.when(mockEvent.getPath()).thenReturn(zkLockPathName);
// notify node deletion
// monitoring should be setup again after event is received
Mockito.when(mockEvent.getType()).thenReturn(Event.EventType.NodeDeleted);
elector.process(mockEvent);
// is standby. no need to notify anything now
Mockito.verify(mockApp, Mockito.times(0)).enterNeutralMode();
// another joinElection called.
Mockito.verify(mockZK, Mockito.times(2)).create(zkLockPathName, data,
Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL, elector, null);
// lost election
elector.processResult(Code.NODEEXISTS.intValue(), zkLockPathName, null,
zkLockPathName);
// still standby. so no need to notify again
Mockito.verify(mockApp, Mockito.times(1)).becomeStandby();
// monitor is set again
Mockito.verify(mockZK, Mockito.times(2)).exists(zkLockPathName, true,
elector, null);
}
/**
* verify quit election terminates connection and there are no new watches.
* next call to joinElection creates new connection and performs election
*/
@Test
public void testQuitElection() throws InterruptedException {
elector.quitElection();
Mockito.verify(mockZK, Mockito.times(1)).close();
// no watches added
Mockito.verify(mockZK, Mockito.times(0)).exists(zkLockPathName, true,
elector, null);
byte[] data = new byte[8];
elector.joinElection(data);
// getNewZooKeeper called 2 times. once in constructor and once now
Assert.assertEquals(2, TestActiveStandbyElector.count);
elector.processResult(Code.NODEEXISTS.intValue(), zkLockPathName, null,
zkLockPathName);
Mockito.verify(mockApp, Mockito.times(1)).becomeStandby();
Mockito.verify(mockZK, Mockito.times(1)).exists(zkLockPathName, true,
elector, null);
}
/**
* verify that receiveActiveData gives data when active exists, tells that
* active does not exist and reports error in getting active information
*
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
* @throws ActiveNotFoundException
*/
@Test
public void testGetActiveData() throws ActiveNotFoundException,
KeeperException, InterruptedException, IOException {
// get valid active data
byte[] data = new byte[8];
Mockito.when(
mockZK.getData(Mockito.eq(zkLockPathName), Mockito.eq(false),
Mockito.<Stat> anyObject())).thenReturn(data);
Assert.assertEquals(data, elector.getActiveData());
Mockito.verify(mockZK, Mockito.times(1)).getData(
Mockito.eq(zkLockPathName), Mockito.eq(false),
Mockito.<Stat> anyObject());
// active does not exist
Mockito.when(
mockZK.getData(Mockito.eq(zkLockPathName), Mockito.eq(false),
Mockito.<Stat> anyObject())).thenThrow(
new KeeperException.NoNodeException());
try {
elector.getActiveData();
Assert.fail("ActiveNotFoundException expected");
} catch(ActiveNotFoundException e) {
Mockito.verify(mockZK, Mockito.times(2)).getData(
Mockito.eq(zkLockPathName), Mockito.eq(false),
Mockito.<Stat> anyObject());
}
// error getting active data rethrows keeperexception
try {
Mockito.when(
mockZK.getData(Mockito.eq(zkLockPathName), Mockito.eq(false),
Mockito.<Stat> anyObject())).thenThrow(
new KeeperException.AuthFailedException());
elector.getActiveData();
Assert.fail("KeeperException.AuthFailedException expected");
} catch(KeeperException.AuthFailedException ke) {
Mockito.verify(mockZK, Mockito.times(3)).getData(
Mockito.eq(zkLockPathName), Mockito.eq(false),
Mockito.<Stat> anyObject());
}
}
}

View File

@ -0,0 +1,231 @@
/**
* 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.ha;
import java.io.File;
import java.io.IOException;
import java.util.List;
import org.junit.Assert;
import org.junit.Test;
import org.apache.hadoop.ha.ActiveStandbyElector.ActiveStandbyElectorCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.test.ClientBase;
/**
* Test for {@link ActiveStandbyElector} using real zookeeper.
*/
public class TestActiveStandbyElectorRealZK extends ClientBase {
static final int NUM_ELECTORS = 2;
static ZooKeeper[] zkClient = new ZooKeeper[NUM_ELECTORS];
static int currentClientIndex = 0;
@Override
public void setUp() throws Exception {
// build.test.dir is used by zookeeper
new File(System.getProperty("build.test.dir", "build")).mkdirs();
super.setUp();
}
class ActiveStandbyElectorTesterRealZK extends ActiveStandbyElector {
ActiveStandbyElectorTesterRealZK(String hostPort, int timeout,
String parent, List<ACL> acl, ActiveStandbyElectorCallback app)
throws IOException {
super(hostPort, timeout, parent, acl, app);
}
@Override
public ZooKeeper getNewZooKeeper() {
return TestActiveStandbyElectorRealZK.zkClient[
TestActiveStandbyElectorRealZK.currentClientIndex];
}
}
/**
* The class object runs on a thread and waits for a signal to start from the
* test object. On getting the signal it joins the election and thus by doing
* this on multiple threads we can test simultaneous attempts at leader lock
* creation. after joining the election, the object waits on a signal to exit.
* this signal comes when the object's elector has become a leader or there is
* an unexpected fatal error. this lets another thread object to become a
* leader.
*/
class ThreadRunner implements Runnable, ActiveStandbyElectorCallback {
int index;
TestActiveStandbyElectorRealZK test;
boolean wait = true;
ThreadRunner(int i, TestActiveStandbyElectorRealZK s) {
index = i;
test = s;
}
@Override
public void run() {
LOG.info("starting " + index);
while(true) {
synchronized (test) {
// wait for test start signal to come
if (!test.start) {
try {
test.wait();
} catch(InterruptedException e) {
Assert.fail(e.getMessage());
}
} else {
break;
}
}
}
// join election
byte[] data = new byte[8];
ActiveStandbyElector elector = test.elector[index];
LOG.info("joining " + index);
elector.joinElection(data);
try {
while(true) {
synchronized (this) {
// wait for elector to become active/fatal error
if (wait) {
// wait to become active
// wait capped at 30s to prevent hung test
wait(30000);
} else {
break;
}
}
}
Thread.sleep(1000);
// quit election to allow other elector to become active
elector.quitElection();
} catch(InterruptedException e) {
Assert.fail(e.getMessage());
}
LOG.info("ending " + index);
}
@Override
public synchronized void becomeActive() {
test.reportActive(index);
LOG.info("active " + index);
wait = false;
notifyAll();
}
@Override
public synchronized void becomeStandby() {
test.reportStandby(index);
LOG.info("standby " + index);
}
@Override
public synchronized void enterNeutralMode() {
LOG.info("neutral " + index);
}
@Override
public synchronized void notifyFatalError(String errorMessage) {
LOG.info("fatal " + index + " .Error message:" + errorMessage);
wait = false;
notifyAll();
}
}
boolean start = false;
int activeIndex = -1;
int standbyIndex = -1;
String parentDir = "/" + java.util.UUID.randomUUID().toString();
ActiveStandbyElector[] elector = new ActiveStandbyElector[NUM_ELECTORS];
ThreadRunner[] threadRunner = new ThreadRunner[NUM_ELECTORS];
Thread[] thread = new Thread[NUM_ELECTORS];
synchronized void reportActive(int index) {
if (activeIndex == -1) {
activeIndex = index;
} else {
// standby should become active
Assert.assertEquals(standbyIndex, index);
// old active should not become active
Assert.assertFalse(activeIndex == index);
}
activeIndex = index;
}
synchronized void reportStandby(int index) {
// only 1 standby should be reported and it should not be the same as active
Assert.assertEquals(-1, standbyIndex);
standbyIndex = index;
Assert.assertFalse(activeIndex == standbyIndex);
}
/**
* the test creates 2 electors which try to become active using a real
* zookeeper server. It verifies that 1 becomes active and 1 becomes standby.
* Upon becoming active the leader quits election and the test verifies that
* the standby now becomes active. these electors run on different threads and
* callback to the test class to report active and standby where the outcome
* is verified
*
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
*/
@Test
public void testActiveStandbyTransition() throws IOException,
InterruptedException, KeeperException {
LOG.info("starting test with parentDir:" + parentDir);
start = false;
byte[] data = new byte[8];
// create random working directory
createClient().create(parentDir, data, Ids.OPEN_ACL_UNSAFE,
CreateMode.PERSISTENT);
for(currentClientIndex = 0;
currentClientIndex < NUM_ELECTORS;
++currentClientIndex) {
LOG.info("creating " + currentClientIndex);
zkClient[currentClientIndex] = createClient();
threadRunner[currentClientIndex] = new ThreadRunner(currentClientIndex,
this);
elector[currentClientIndex] = new ActiveStandbyElectorTesterRealZK(
"hostPort", 1000, parentDir, Ids.OPEN_ACL_UNSAFE,
threadRunner[currentClientIndex]);
zkClient[currentClientIndex].register(elector[currentClientIndex]);
thread[currentClientIndex] = new Thread(threadRunner[currentClientIndex]);
thread[currentClientIndex].start();
}
synchronized (this) {
// signal threads to start
LOG.info("signaling threads");
start = true;
notifyAll();
}
for(int i = 0; i < thread.length; i++) {
thread[i].join();
}
}
}

View File

@ -0,0 +1,441 @@
/**
* 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.ha;
import java.io.IOException;
import java.net.InetSocketAddress;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.verify;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.ha.protocolPB.HAServiceProtocolClientSideTranslatorPB;
import org.apache.hadoop.ha.TestNodeFencer.AlwaysSucceedFencer;
import org.apache.hadoop.ha.TestNodeFencer.AlwaysFailFencer;
import static org.apache.hadoop.ha.TestNodeFencer.setupFencer;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.AccessControlException;
import org.junit.Test;
import static org.junit.Assert.*;
public class TestFailoverController {
private InetSocketAddress svc1Addr = new InetSocketAddress("svc1", 1234);
private InetSocketAddress svc2Addr = new InetSocketAddress("svc2", 5678);
private class DummyService implements HAServiceProtocol {
HAServiceState state;
DummyService(HAServiceState state) {
this.state = state;
}
@Override
public void monitorHealth() throws HealthCheckFailedException, IOException {
// Do nothing
}
@Override
public void transitionToActive() throws ServiceFailedException, IOException {
state = HAServiceState.ACTIVE;
}
@Override
public void transitionToStandby() throws ServiceFailedException, IOException {
state = HAServiceState.STANDBY;
}
@Override
public HAServiceState getServiceState() throws IOException {
return state;
}
@Override
public boolean readyToBecomeActive() throws ServiceFailedException, IOException {
return true;
}
}
@Test
public void testFailoverAndFailback() throws Exception {
DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
DummyService svc2 = new DummyService(HAServiceState.STANDBY);
NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
AlwaysSucceedFencer.fenceCalled = 0;
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, false);
assertEquals(0, TestNodeFencer.AlwaysSucceedFencer.fenceCalled);
assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
AlwaysSucceedFencer.fenceCalled = 0;
FailoverController.failover(svc2, svc2Addr, svc1, svc1Addr, fencer, false, false);
assertEquals(0, TestNodeFencer.AlwaysSucceedFencer.fenceCalled);
assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
}
@Test
public void testFailoverFromStandbyToStandby() throws Exception {
DummyService svc1 = new DummyService(HAServiceState.STANDBY);
DummyService svc2 = new DummyService(HAServiceState.STANDBY);
NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, false);
assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
}
@Test
public void testFailoverFromActiveToActive() throws Exception {
DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
DummyService svc2 = new DummyService(HAServiceState.ACTIVE);
NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
try {
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, false);
fail("Can't failover to an already active service");
} catch (FailoverFailedException ffe) {
// Expected
}
assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
}
@Test
public void testFailoverWithoutPermission() throws Exception {
DummyService svc1 = new DummyService(HAServiceState.ACTIVE) {
@Override
public HAServiceState getServiceState() throws IOException {
throw new AccessControlException("Access denied");
}
};
DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
@Override
public HAServiceState getServiceState() throws IOException {
throw new AccessControlException("Access denied");
}
};
NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
try {
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, false);
fail("Can't failover when access is denied");
} catch (FailoverFailedException ffe) {
assertTrue(ffe.getCause().getMessage().contains("Access denied"));
}
}
@Test
public void testFailoverToUnreadyService() throws Exception {
DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
@Override
public boolean readyToBecomeActive() throws ServiceFailedException, IOException {
return false;
}
};
NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
try {
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, false);
fail("Can't failover to a service that's not ready");
} catch (FailoverFailedException ffe) {
// Expected
}
assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
// Forcing it means we ignore readyToBecomeActive
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, true);
assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
}
@Test
public void testFailoverToUnhealthyServiceFailsAndFailsback() throws Exception {
DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
@Override
public void monitorHealth() throws HealthCheckFailedException {
throw new HealthCheckFailedException("Failed!");
}
};
NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
try {
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, false);
fail("Failover to unhealthy service");
} catch (FailoverFailedException ffe) {
// Expected
}
assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
}
@Test
public void testFailoverFromFaultyServiceSucceeds() throws Exception {
DummyService svc1 = new DummyService(HAServiceState.ACTIVE) {
@Override
public void transitionToStandby() throws ServiceFailedException {
throw new ServiceFailedException("Failed!");
}
};
DummyService svc2 = new DummyService(HAServiceState.STANDBY);
NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
AlwaysSucceedFencer.fenceCalled = 0;
try {
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, false);
} catch (FailoverFailedException ffe) {
fail("Faulty active prevented failover");
}
// svc1 still thinks it's active, that's OK, it was fenced
assertEquals(1, AlwaysSucceedFencer.fenceCalled);
assertEquals("svc1:1234", AlwaysSucceedFencer.fencedSvc);
assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
}
@Test
public void testFailoverFromFaultyServiceFencingFailure() throws Exception {
DummyService svc1 = new DummyService(HAServiceState.ACTIVE) {
@Override
public void transitionToStandby() throws ServiceFailedException {
throw new ServiceFailedException("Failed!");
}
};
DummyService svc2 = new DummyService(HAServiceState.STANDBY);
NodeFencer fencer = setupFencer(AlwaysFailFencer.class.getName());
AlwaysFailFencer.fenceCalled = 0;
try {
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, false);
fail("Failed over even though fencing failed");
} catch (FailoverFailedException ffe) {
// Expected
}
assertEquals(1, AlwaysFailFencer.fenceCalled);
assertEquals("svc1:1234", AlwaysFailFencer.fencedSvc);
assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
}
@Test
public void testFencingFailureDuringFailover() throws Exception {
DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
DummyService svc2 = new DummyService(HAServiceState.STANDBY);
NodeFencer fencer = setupFencer(AlwaysFailFencer.class.getName());
AlwaysFailFencer.fenceCalled = 0;
try {
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, true, false);
fail("Failed over even though fencing requested and failed");
} catch (FailoverFailedException ffe) {
// Expected
}
// If fencing was requested and it failed we don't try to make
// svc2 active anyway, and we don't failback to svc1.
assertEquals(1, AlwaysFailFencer.fenceCalled);
assertEquals("svc1:1234", AlwaysFailFencer.fencedSvc);
assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
}
private HAServiceProtocol getProtocol(String target)
throws IOException {
InetSocketAddress addr = NetUtils.createSocketAddr(target);
Configuration conf = new Configuration();
// Lower the timeout so we quickly fail to connect
conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 1);
return new HAServiceProtocolClientSideTranslatorPB(addr, conf);
}
@Test
public void testFailoverFromNonExistantServiceWithFencer() throws Exception {
HAServiceProtocol svc1 = getProtocol("localhost:1234");
DummyService svc2 = new DummyService(HAServiceState.STANDBY);
NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
try {
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, false);
} catch (FailoverFailedException ffe) {
fail("Non-existant active prevented failover");
}
// Don't check svc1 because we can't reach it, but that's OK, it's been fenced.
assertEquals(HAServiceState.ACTIVE, svc2.getServiceState());
}
@Test
public void testFailoverToNonExistantServiceFails() throws Exception {
DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
HAServiceProtocol svc2 = getProtocol("localhost:1234");
NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
try {
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, false);
fail("Failed over to a non-existant standby");
} catch (FailoverFailedException ffe) {
// Expected
}
assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
}
@Test
public void testFailoverToFaultyServiceFailsbackOK() throws Exception {
DummyService svc1 = spy(new DummyService(HAServiceState.ACTIVE));
DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
@Override
public void transitionToActive() throws ServiceFailedException {
throw new ServiceFailedException("Failed!");
}
};
NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
try {
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, false);
fail("Failover to already active service");
} catch (FailoverFailedException ffe) {
// Expected
}
// svc1 went standby then back to active
verify(svc1).transitionToStandby();
verify(svc1).transitionToActive();
assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
}
@Test
public void testWeDontFailbackIfActiveWasFenced() throws Exception {
DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
@Override
public void transitionToActive() throws ServiceFailedException {
throw new ServiceFailedException("Failed!");
}
};
NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
try {
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, true, false);
fail("Failed over to service that won't transition to active");
} catch (FailoverFailedException ffe) {
// Expected
}
// We failed to failover and did not failback because we fenced
// svc1 (we forced it), therefore svc1 and svc2 should be standby.
assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
}
@Test
public void testWeFenceOnFailbackIfTransitionToActiveFails() throws Exception {
DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
@Override
public void transitionToActive() throws ServiceFailedException, IOException {
throw new IOException("Failed!");
}
};
NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
AlwaysSucceedFencer.fenceCalled = 0;
try {
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, false);
fail("Failed over to service that won't transition to active");
} catch (FailoverFailedException ffe) {
// Expected
}
// We failed to failover. We did not fence svc1 because it cooperated
// and we didn't force it, so we failed back to svc1 and fenced svc2.
// Note svc2 still thinks it's active, that's OK, we fenced it.
assertEquals(HAServiceState.ACTIVE, svc1.getServiceState());
assertEquals(1, AlwaysSucceedFencer.fenceCalled);
assertEquals("svc2:5678", AlwaysSucceedFencer.fencedSvc);
}
@Test
public void testFailureToFenceOnFailbackFailsTheFailback() throws Exception {
DummyService svc1 = new DummyService(HAServiceState.ACTIVE);
DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
@Override
public void transitionToActive() throws ServiceFailedException, IOException {
throw new IOException("Failed!");
}
};
NodeFencer fencer = setupFencer(AlwaysFailFencer.class.getName());
AlwaysFailFencer.fenceCalled = 0;
try {
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, false);
fail("Failed over to service that won't transition to active");
} catch (FailoverFailedException ffe) {
// Expected
}
// We did not fence svc1 because it cooperated and we didn't force it,
// we failed to failover so we fenced svc2, we failed to fence svc2
// so we did not failback to svc1, ie it's still standby.
assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
assertEquals(1, AlwaysFailFencer.fenceCalled);
assertEquals("svc2:5678", AlwaysFailFencer.fencedSvc);
}
@Test
public void testFailbackToFaultyServiceFails() throws Exception {
DummyService svc1 = new DummyService(HAServiceState.ACTIVE) {
@Override
public void transitionToActive() throws ServiceFailedException {
throw new ServiceFailedException("Failed!");
}
};
DummyService svc2 = new DummyService(HAServiceState.STANDBY) {
@Override
public void transitionToActive() throws ServiceFailedException {
throw new ServiceFailedException("Failed!");
}
};
NodeFencer fencer = setupFencer(AlwaysSucceedFencer.class.getName());
try {
FailoverController.failover(svc1, svc1Addr, svc2, svc2Addr, fencer, false, false);
fail("Failover to already active service");
} catch (FailoverFailedException ffe) {
// Expected
}
assertEquals(HAServiceState.STANDBY, svc1.getServiceState());
assertEquals(HAServiceState.STANDBY, svc2.getServiceState());
}
}

View File

@ -0,0 +1,106 @@
/**
* 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.ha;
import static org.junit.Assert.*;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.PrintStream;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
import static org.mockito.Mockito.when;
import com.google.common.base.Charsets;
import com.google.common.base.Joiner;
public class TestHAAdmin {
private static final Log LOG = LogFactory.getLog(TestHAAdmin.class);
private HAAdmin tool;
private ByteArrayOutputStream errOutBytes = new ByteArrayOutputStream();
private String errOutput;
private HAServiceProtocol mockProtocol;
@Before
public void setup() throws IOException {
mockProtocol = Mockito.mock(HAServiceProtocol.class);
when(mockProtocol.readyToBecomeActive()).thenReturn(true);
tool = new HAAdmin() {
@Override
protected HAServiceProtocol getProtocol(String target) throws IOException {
return mockProtocol;
}
};
tool.setConf(new Configuration());
tool.errOut = new PrintStream(errOutBytes);
}
private void assertOutputContains(String string) {
if (!errOutput.contains(string)) {
fail("Expected output to contain '" + string + "' but was:\n" +
errOutput);
}
}
@Test
public void testAdminUsage() throws Exception {
assertEquals(-1, runTool());
assertOutputContains("Usage:");
assertOutputContains("-transitionToActive");
assertEquals(-1, runTool("badCommand"));
assertOutputContains("Bad command 'badCommand'");
assertEquals(-1, runTool("-badCommand"));
assertOutputContains("badCommand: Unknown");
// valid command but not enough arguments
assertEquals(-1, runTool("-transitionToActive"));
assertOutputContains("transitionToActive: incorrect number of arguments");
assertEquals(-1, runTool("-transitionToActive", "x", "y"));
assertOutputContains("transitionToActive: incorrect number of arguments");
assertEquals(-1, runTool("-failover"));
assertOutputContains("failover: incorrect arguments");
assertOutputContains("failover: incorrect arguments");
assertEquals(-1, runTool("-failover", "foo:1234"));
assertOutputContains("failover: incorrect arguments");
}
@Test
public void testHelp() throws Exception {
assertEquals(-1, runTool("-help"));
assertEquals(0, runTool("-help", "transitionToActive"));
assertOutputContains("Transitions the service into Active");
}
private Object runTool(String ... args) throws Exception {
errOutBytes.reset();
LOG.info("Running: HAAdmin " + Joiner.on(" ").join(args));
int ret = tool.run(args);
errOutput = new String(errOutBytes.toByteArray(), Charsets.UTF_8);
LOG.info("Output:\n" + errOutput);
return ret;
}
}

View File

@ -0,0 +1,173 @@
/**
* 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.ha;
import static org.junit.Assert.*;
import java.net.InetSocketAddress;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.junit.Before;
import org.junit.Test;
import com.google.common.collect.Lists;
public class TestNodeFencer {
@Before
public void clearMockState() {
AlwaysSucceedFencer.fenceCalled = 0;
AlwaysSucceedFencer.callArgs.clear();
AlwaysFailFencer.fenceCalled = 0;
AlwaysFailFencer.callArgs.clear();
}
@Test
public void testSingleFencer() throws BadFencingConfigurationException {
NodeFencer fencer = setupFencer(
AlwaysSucceedFencer.class.getName() + "(foo)");
assertTrue(fencer.fence(new InetSocketAddress("host", 1234)));
assertEquals(1, AlwaysSucceedFencer.fenceCalled);
assertEquals("host:1234", AlwaysSucceedFencer.fencedSvc);
assertEquals("foo", AlwaysSucceedFencer.callArgs.get(0));
}
@Test
public void testMultipleFencers() throws BadFencingConfigurationException {
NodeFencer fencer = setupFencer(
AlwaysSucceedFencer.class.getName() + "(foo)\n" +
AlwaysSucceedFencer.class.getName() + "(bar)\n");
assertTrue(fencer.fence(new InetSocketAddress("host", 1234)));
// Only one call, since the first fencer succeeds
assertEquals(1, AlwaysSucceedFencer.fenceCalled);
assertEquals("foo", AlwaysSucceedFencer.callArgs.get(0));
}
@Test
public void testWhitespaceAndCommentsInConfig()
throws BadFencingConfigurationException {
NodeFencer fencer = setupFencer(
"\n" +
" # the next one will always fail\n" +
" " + AlwaysFailFencer.class.getName() + "(foo) # <- fails\n" +
AlwaysSucceedFencer.class.getName() + "(bar) \n");
assertTrue(fencer.fence(new InetSocketAddress("host", 1234)));
// One call to each, since top fencer fails
assertEquals(1, AlwaysFailFencer.fenceCalled);
assertEquals("host:1234", AlwaysFailFencer.fencedSvc);
assertEquals(1, AlwaysSucceedFencer.fenceCalled);
assertEquals("host:1234", AlwaysSucceedFencer.fencedSvc);
assertEquals("foo", AlwaysFailFencer.callArgs.get(0));
assertEquals("bar", AlwaysSucceedFencer.callArgs.get(0));
}
@Test
public void testArglessFencer() throws BadFencingConfigurationException {
NodeFencer fencer = setupFencer(
AlwaysSucceedFencer.class.getName());
assertTrue(fencer.fence(new InetSocketAddress("host", 1234)));
// One call to each, since top fencer fails
assertEquals(1, AlwaysSucceedFencer.fenceCalled);
assertEquals("host:1234", AlwaysSucceedFencer.fencedSvc);
assertEquals(null, AlwaysSucceedFencer.callArgs.get(0));
}
@Test
public void testShortNameShell() throws BadFencingConfigurationException {
NodeFencer fencer = setupFencer("shell(true)");
assertTrue(fencer.fence(new InetSocketAddress("host", 1234)));
}
@Test
public void testShortNameSsh() throws BadFencingConfigurationException {
NodeFencer fencer = setupFencer("sshfence");
assertFalse(fencer.fence(new InetSocketAddress("host", 1234)));
}
@Test
public void testShortNameSshWithUser() throws BadFencingConfigurationException {
NodeFencer fencer = setupFencer("sshfence(user)");
assertFalse(fencer.fence(new InetSocketAddress("host", 1234)));
}
@Test
public void testShortNameSshWithPort() throws BadFencingConfigurationException {
NodeFencer fencer = setupFencer("sshfence(:123)");
assertFalse(fencer.fence(new InetSocketAddress("host", 1234)));
}
@Test
public void testShortNameSshWithUserPort() throws BadFencingConfigurationException {
NodeFencer fencer = setupFencer("sshfence(user:123)");
assertFalse(fencer.fence(new InetSocketAddress("host", 1234)));
}
public static NodeFencer setupFencer(String confStr)
throws BadFencingConfigurationException {
System.err.println("Testing configuration:\n" + confStr);
Configuration conf = new Configuration();
conf.set(NodeFencer.CONF_METHODS_KEY, confStr);
return new NodeFencer(conf);
}
/**
* Mock fencing method that always returns true
*/
public static class AlwaysSucceedFencer extends Configured
implements FenceMethod {
static int fenceCalled = 0;
static String fencedSvc;
static List<String> callArgs = Lists.newArrayList();
@Override
public boolean tryFence(InetSocketAddress serviceAddr, String args) {
fencedSvc = serviceAddr.getHostName() + ":" + serviceAddr.getPort();
callArgs.add(args);
fenceCalled++;
return true;
}
@Override
public void checkArgs(String args) {
}
}
/**
* Identical mock to above, except always returns false
*/
public static class AlwaysFailFencer extends Configured
implements FenceMethod {
static int fenceCalled = 0;
static String fencedSvc;
static List<String> callArgs = Lists.newArrayList();
@Override
public boolean tryFence(InetSocketAddress serviceAddr, String args) {
fencedSvc = serviceAddr.getHostName() + ":" + serviceAddr.getPort();
callArgs.add(args);
fenceCalled++;
return false;
}
@Override
public void checkArgs(String args) {
}
}
}

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.ha;
import static org.junit.Assert.*;
import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.StringUtils;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.mockito.Mockito;
import static org.mockito.Mockito.spy;
public class TestShellCommandFencer {
private ShellCommandFencer fencer = createFencer();
@BeforeClass
public static void setupLogSpy() {
ShellCommandFencer.LOG = spy(ShellCommandFencer.LOG);
}
@Before
public void resetLogSpy() {
Mockito.reset(ShellCommandFencer.LOG);
}
private static ShellCommandFencer createFencer() {
Configuration conf = new Configuration();
conf.set("in.fencing.tests", "yessir");
ShellCommandFencer fencer = new ShellCommandFencer();
fencer.setConf(conf);
return fencer;
}
/**
* Test that the exit code of the script determines
* whether the fencer succeeded or failed
*/
@Test
public void testBasicSuccessFailure() {
InetSocketAddress addr = new InetSocketAddress("host", 1234);
assertTrue(fencer.tryFence(addr, "echo"));
assertFalse(fencer.tryFence(addr, "exit 1"));
// bad path should also fail
assertFalse(fencer.tryFence(addr, "xxxxxxxxxxxx"));
}
@Test
public void testCheckNoArgs() {
try {
Configuration conf = new Configuration();
conf.set(NodeFencer.CONF_METHODS_KEY, "shell");
new NodeFencer(conf);
fail("Didn't throw when passing no args to shell");
} catch (BadFencingConfigurationException confe) {
assertTrue(
"Unexpected exception:" + StringUtils.stringifyException(confe),
confe.getMessage().contains("No argument passed"));
}
}
@Test
public void testCheckParensNoArgs() {
try {
Configuration conf = new Configuration();
conf.set(NodeFencer.CONF_METHODS_KEY, "shell()");
new NodeFencer(conf);
fail("Didn't throw when passing no args to shell");
} catch (BadFencingConfigurationException confe) {
assertTrue(
"Unexpected exception:" + StringUtils.stringifyException(confe),
confe.getMessage().contains("Unable to parse line: 'shell()'"));
}
}
/**
* Test that lines on stdout get passed as INFO
* level messages
*/
@Test
public void testStdoutLogging() {
InetSocketAddress addr = new InetSocketAddress("host", 1234);
assertTrue(fencer.tryFence(addr, "echo hello"));
Mockito.verify(ShellCommandFencer.LOG).info(
Mockito.endsWith("echo hello: host:1234 hello"));
}
/**
* Test that lines on stderr get passed as
* WARN level log messages
*/
@Test
public void testStderrLogging() {
InetSocketAddress addr = new InetSocketAddress("host", 1234);
assertTrue(fencer.tryFence(addr, "echo hello >&2"));
Mockito.verify(ShellCommandFencer.LOG).warn(
Mockito.endsWith("echo hello >&2: host:1234 hello"));
}
/**
* Verify that the Configuration gets passed as
* environment variables to the fencer.
*/
@Test
public void testConfAsEnvironment() {
InetSocketAddress addr = new InetSocketAddress("host", 1234);
fencer.tryFence(addr, "echo $in_fencing_tests");
Mockito.verify(ShellCommandFencer.LOG).info(
Mockito.endsWith("echo $in...ing_tests: host:1234 yessir"));
}
/**
* Test that we properly close off our input to the subprocess
* such that it knows there's no tty connected. This is important
* so that, if we use 'ssh', it won't try to prompt for a password
* and block forever, for example.
*/
@Test(timeout=10000)
public void testSubprocessInputIsClosed() {
InetSocketAddress addr = new InetSocketAddress("host", 1234);
assertFalse(fencer.tryFence(addr, "read"));
}
@Test
public void testCommandAbbreviation() {
assertEquals("a...f", ShellCommandFencer.abbreviate("abcdef", 5));
assertEquals("abcdef", ShellCommandFencer.abbreviate("abcdef", 6));
assertEquals("abcdef", ShellCommandFencer.abbreviate("abcdef", 7));
assertEquals("a...g", ShellCommandFencer.abbreviate("abcdefg", 5));
assertEquals("a...h", ShellCommandFencer.abbreviate("abcdefgh", 5));
assertEquals("a...gh", ShellCommandFencer.abbreviate("abcdefgh", 6));
assertEquals("ab...gh", ShellCommandFencer.abbreviate("abcdefgh", 7));
}
}

View File

@ -0,0 +1,130 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ha;
import static org.junit.Assert.*;
import java.net.InetSocketAddress;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ha.SshFenceByTcpPort.Args;
import org.apache.log4j.Level;
import org.junit.Assume;
import org.junit.Test;
public class TestSshFenceByTcpPort {
static {
((Log4JLogger)SshFenceByTcpPort.LOG).getLogger().setLevel(Level.ALL);
}
private String TEST_FENCING_HOST = System.getProperty(
"test.TestSshFenceByTcpPort.host", "localhost");
private String TEST_FENCING_PORT = System.getProperty(
"test.TestSshFenceByTcpPort.port", "8020");
private final String TEST_KEYFILE = System.getProperty(
"test.TestSshFenceByTcpPort.key");
@Test(timeout=20000)
public void testFence() throws BadFencingConfigurationException {
Assume.assumeTrue(isConfigured());
Configuration conf = new Configuration();
conf.set(SshFenceByTcpPort.CONF_IDENTITIES_KEY, TEST_KEYFILE);
SshFenceByTcpPort fence = new SshFenceByTcpPort();
fence.setConf(conf);
assertTrue(fence.tryFence(
new InetSocketAddress(TEST_FENCING_HOST,
Integer.valueOf(TEST_FENCING_PORT)),
null));
}
/**
* Test connecting to a host which definitely won't respond.
* Make sure that it times out and returns false, but doesn't throw
* any exception
*/
@Test(timeout=20000)
public void testConnectTimeout() throws BadFencingConfigurationException {
Configuration conf = new Configuration();
conf.setInt(SshFenceByTcpPort.CONF_CONNECT_TIMEOUT_KEY, 3000);
SshFenceByTcpPort fence = new SshFenceByTcpPort();
fence.setConf(conf);
// Connect to Google's DNS server - not running ssh!
assertFalse(fence.tryFence(new InetSocketAddress("8.8.8.8", 1234), ""));
}
@Test
public void testArgsParsing() throws BadFencingConfigurationException {
InetSocketAddress addr = new InetSocketAddress("bar.com", 1234);
Args args = new SshFenceByTcpPort.Args(addr, null);
assertEquals("bar.com", args.host);
assertEquals(1234, args.targetPort);
assertEquals(System.getProperty("user.name"), args.user);
assertEquals(22, args.sshPort);
args = new SshFenceByTcpPort.Args(addr, "");
assertEquals("bar.com", args.host);
assertEquals(1234, args.targetPort);
assertEquals(System.getProperty("user.name"), args.user);
assertEquals(22, args.sshPort);
args = new SshFenceByTcpPort.Args(addr, "12345");
assertEquals("bar.com", args.host);
assertEquals(1234, args.targetPort);
assertEquals("12345", args.user);
assertEquals(22, args.sshPort);
args = new SshFenceByTcpPort.Args(addr, ":12345");
assertEquals("bar.com", args.host);
assertEquals(1234, args.targetPort);
assertEquals(System.getProperty("user.name"), args.user);
assertEquals(12345, args.sshPort);
args = new SshFenceByTcpPort.Args(addr, "foo:8020");
assertEquals("bar.com", args.host);
assertEquals(1234, args.targetPort);
assertEquals("foo", args.user);
assertEquals(8020, args.sshPort);
}
@Test
public void testBadArgsParsing() throws BadFencingConfigurationException {
assertBadArgs(":"); // No port specified
assertBadArgs("bar.com:"); // "
assertBadArgs(":xx"); // Port does not parse
assertBadArgs("bar.com:xx"); // "
}
private void assertBadArgs(String argStr) {
InetSocketAddress addr = new InetSocketAddress("bar.com", 1234);
try {
new Args(addr, argStr);
fail("Did not fail on bad args: " + argStr);
} catch (BadFencingConfigurationException e) {
// Expected
}
}
private boolean isConfigured() {
return (TEST_FENCING_HOST != null && !TEST_FENCING_HOST.isEmpty()) &&
(TEST_FENCING_PORT != null && !TEST_FENCING_PORT.isEmpty()) &&
(TEST_KEYFILE != null && !TEST_KEYFILE.isEmpty());
}
}

View File

@ -25,21 +25,23 @@ import java.util.concurrent.CountDownLatch;
import org.apache.hadoop.io.retry.UnreliableImplementation.TypeOfExceptionToFailWith;
import org.apache.hadoop.io.retry.UnreliableInterface.UnreliableException;
import org.apache.hadoop.ipc.StandbyException;
import org.apache.hadoop.util.ThreadUtil;
import org.junit.Test;
@SuppressWarnings("unchecked")
public class TestFailoverProxy {
public static class FlipFlopProxyProvider implements FailoverProxyProvider {
public static class FlipFlopProxyProvider<T> implements FailoverProxyProvider<T> {
private Class<?> iface;
private Object currentlyActive;
private Object impl1;
private Object impl2;
private Class<T> iface;
private T currentlyActive;
private T impl1;
private T impl2;
private int failoversOccurred = 0;
public FlipFlopProxyProvider(Class<?> iface, Object activeImpl,
Object standbyImpl) {
public FlipFlopProxyProvider(Class<T> iface, T activeImpl,
T standbyImpl) {
this.iface = iface;
this.impl1 = activeImpl;
this.impl2 = standbyImpl;
@ -47,7 +49,7 @@ public class TestFailoverProxy {
}
@Override
public Object getProxy() {
public T getProxy() {
return currentlyActive;
}
@ -58,7 +60,7 @@ public class TestFailoverProxy {
}
@Override
public Class<?> getInterface() {
public Class<T> getInterface() {
return iface;
}
@ -126,7 +128,7 @@ public class TestFailoverProxy {
new FlipFlopProxyProvider(UnreliableInterface.class,
new UnreliableImplementation("impl1"),
new UnreliableImplementation("impl2")),
RetryPolicies.TRY_ONCE_DONT_FAIL);
RetryPolicies.TRY_ONCE_THEN_FAIL);
unreliable.succeedsOnceThenFailsReturningString();
try {
@ -180,7 +182,7 @@ public class TestFailoverProxy {
assertEquals("impl1", unreliable.succeedsOnceThenFailsReturningString());
try {
assertEquals("impl2", unreliable.succeedsOnceThenFailsReturningString());
unreliable.succeedsOnceThenFailsReturningString();
fail("should not have succeeded twice");
} catch (IOException e) {
// Make sure we *don't* fail over since the first implementation threw an
@ -194,6 +196,27 @@ public class TestFailoverProxy {
assertEquals("impl2", unreliable.succeedsOnceThenFailsReturningStringIdempotent());
}
/**
* Test that if a non-idempotent void function is called, and there is an exception,
* the exception is properly propagated
*/
@Test
public void testExceptionPropagatedForNonIdempotentVoid() throws Exception {
UnreliableInterface unreliable = (UnreliableInterface)RetryProxy
.create(UnreliableInterface.class,
new FlipFlopProxyProvider(UnreliableInterface.class,
new UnreliableImplementation("impl1", TypeOfExceptionToFailWith.IO_EXCEPTION),
new UnreliableImplementation("impl2", TypeOfExceptionToFailWith.UNRELIABLE_EXCEPTION)),
RetryPolicies.failoverOnNetworkException(1));
try {
unreliable.nonIdempotentVoidFailsIfIdentifierDoesntMatch("impl2");
fail("did not throw an exception");
} catch (Exception e) {
}
}
private static class SynchronizedUnreliableImplementation extends UnreliableImplementation {
private CountDownLatch methodLatch;
@ -267,4 +290,62 @@ public class TestFailoverProxy {
assertEquals("impl2", t2.result);
assertEquals(1, proxyProvider.getFailoversOccurred());
}
/**
* Ensure that when all configured services are throwing StandbyException
* that we fail over back and forth between them until one is no longer
* throwing StandbyException.
*/
@Test
public void testFailoverBetweenMultipleStandbys()
throws UnreliableException, StandbyException, IOException {
final long millisToSleep = 10000;
final UnreliableImplementation impl1 = new UnreliableImplementation("impl1",
TypeOfExceptionToFailWith.STANDBY_EXCEPTION);
FlipFlopProxyProvider proxyProvider = new FlipFlopProxyProvider(
UnreliableInterface.class,
impl1,
new UnreliableImplementation("impl2",
TypeOfExceptionToFailWith.STANDBY_EXCEPTION));
final UnreliableInterface unreliable = (UnreliableInterface)RetryProxy
.create(UnreliableInterface.class, proxyProvider,
RetryPolicies.failoverOnNetworkException(
RetryPolicies.TRY_ONCE_THEN_FAIL, 10, 1000, 10000));
new Thread() {
@Override
public void run() {
ThreadUtil.sleepAtLeastIgnoreInterrupts(millisToSleep);
impl1.setIdentifier("renamed-impl1");
}
}.start();
String result = unreliable.failsIfIdentifierDoesntMatch("renamed-impl1");
assertEquals("renamed-impl1", result);
}
/**
* Ensure that normal IO exceptions don't result in a failover.
*/
@Test
public void testExpectedIOException() {
UnreliableInterface unreliable = (UnreliableInterface)RetryProxy
.create(UnreliableInterface.class,
new FlipFlopProxyProvider(UnreliableInterface.class,
new UnreliableImplementation("impl1", TypeOfExceptionToFailWith.REMOTE_EXCEPTION),
new UnreliableImplementation("impl2", TypeOfExceptionToFailWith.UNRELIABLE_EXCEPTION)),
RetryPolicies.failoverOnNetworkException(
RetryPolicies.TRY_ONCE_THEN_FAIL, 10, 1000, 10000));
try {
unreliable.failsIfIdentifierDoesntMatch("no-such-identifier");
fail("Should have thrown *some* exception");
} catch (Exception e) {
assertTrue("Expected IOE but got " + e.getClass(),
e instanceof IOException);
}
}
}

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.io.retry;
import static org.apache.hadoop.io.retry.RetryPolicies.RETRY_FOREVER;
import static org.apache.hadoop.io.retry.RetryPolicies.TRY_ONCE_DONT_FAIL;
import static org.apache.hadoop.io.retry.RetryPolicies.TRY_ONCE_THEN_FAIL;
import static org.apache.hadoop.io.retry.RetryPolicies.retryByException;
import static org.apache.hadoop.io.retry.RetryPolicies.retryByRemoteException;
@ -59,19 +58,6 @@ public class TestRetryProxy extends TestCase {
}
}
public void testTryOnceDontFail() throws UnreliableException {
UnreliableInterface unreliable = (UnreliableInterface)
RetryProxy.create(UnreliableInterface.class, unreliableImpl, TRY_ONCE_DONT_FAIL);
unreliable.alwaysSucceeds();
unreliable.failsOnceThenSucceeds();
try {
unreliable.failsOnceThenSucceedsWithReturnValue();
fail("Should fail");
} catch (UnreliableException e) {
// expected
}
}
public void testRetryForever() throws UnreliableException {
UnreliableInterface unreliable = (UnreliableInterface)
RetryProxy.create(UnreliableInterface.class, unreliableImpl, RETRY_FOREVER);

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.io.retry;
import java.io.IOException;
import org.apache.hadoop.io.retry.UnreliableInterface.UnreliableException;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.StandbyException;
@ -37,7 +38,8 @@ public class UnreliableImplementation implements UnreliableInterface {
public static enum TypeOfExceptionToFailWith {
UNRELIABLE_EXCEPTION,
STANDBY_EXCEPTION,
IO_EXCEPTION
IO_EXCEPTION,
REMOTE_EXCEPTION
}
public UnreliableImplementation() {
@ -48,6 +50,10 @@ public class UnreliableImplementation implements UnreliableInterface {
this(identifier, TypeOfExceptionToFailWith.UNRELIABLE_EXCEPTION);
}
public void setIdentifier(String identifier) {
this.identifier = identifier;
}
public UnreliableImplementation(String identifier,
TypeOfExceptionToFailWith exceptionToFailWith) {
this.identifier = identifier;
@ -91,14 +97,7 @@ public class UnreliableImplementation implements UnreliableInterface {
if (succeedsOnceThenFailsCount++ < 1) {
return identifier;
} else {
switch (exceptionToFailWith) {
case STANDBY_EXCEPTION:
throw new StandbyException(identifier);
case UNRELIABLE_EXCEPTION:
throw new UnreliableException(identifier);
case IO_EXCEPTION:
throw new IOException(identifier);
}
throwAppropriateException(exceptionToFailWith, identifier);
return null;
}
}
@ -109,16 +108,8 @@ public class UnreliableImplementation implements UnreliableInterface {
if (succeedsTenTimesThenFailsCount++ < 10) {
return identifier;
} else {
switch (exceptionToFailWith) {
case STANDBY_EXCEPTION:
throw new StandbyException(identifier);
case UNRELIABLE_EXCEPTION:
throw new UnreliableException(identifier);
case IO_EXCEPTION:
throw new IOException(identifier);
default:
throw new RuntimeException(identifier);
}
throwAppropriateException(exceptionToFailWith, identifier);
return null;
}
}
@ -128,16 +119,8 @@ public class UnreliableImplementation implements UnreliableInterface {
if (succeedsOnceThenFailsIdempotentCount++ < 1) {
return identifier;
} else {
switch (exceptionToFailWith) {
case STANDBY_EXCEPTION:
throw new StandbyException(identifier);
case UNRELIABLE_EXCEPTION:
throw new UnreliableException(identifier);
case IO_EXCEPTION:
throw new IOException(identifier);
default:
throw new RuntimeException(identifier);
}
throwAppropriateException(exceptionToFailWith, identifier);
return null;
}
}
@ -147,17 +130,38 @@ public class UnreliableImplementation implements UnreliableInterface {
if (this.identifier.equals(identifier)) {
return identifier;
} else {
switch (exceptionToFailWith) {
case STANDBY_EXCEPTION:
throw new StandbyException(identifier);
case UNRELIABLE_EXCEPTION:
throw new UnreliableException(identifier);
case IO_EXCEPTION:
throw new IOException(identifier);
default:
throw new RuntimeException(identifier);
}
String message = "expected '" + this.identifier + "' but received '" +
identifier + "'";
throwAppropriateException(exceptionToFailWith, message);
return null;
}
}
@Override
public void nonIdempotentVoidFailsIfIdentifierDoesntMatch(String identifier)
throws UnreliableException, StandbyException, IOException {
if (this.identifier.equals(identifier)) {
return;
} else {
String message = "expected '" + this.identifier + "' but received '" +
identifier + "'";
throwAppropriateException(exceptionToFailWith, message);
}
}
private static void throwAppropriateException(TypeOfExceptionToFailWith eType,
String message) throws UnreliableException, StandbyException, IOException {
switch (eType) {
case STANDBY_EXCEPTION:
throw new StandbyException(message);
case UNRELIABLE_EXCEPTION:
throw new UnreliableException(message);
case IO_EXCEPTION:
throw new IOException(message);
case REMOTE_EXCEPTION:
throw new RemoteException(IOException.class.getName(), message);
default:
throw new RuntimeException(message);
}
}
}

View File

@ -67,4 +67,7 @@ public interface UnreliableInterface {
@Idempotent
public String failsIfIdentifierDoesntMatch(String identifier)
throws UnreliableException, StandbyException, IOException;
void nonIdempotentVoidFailsIfIdentifierDoesntMatch(String identifier)
throws UnreliableException, StandbyException, IOException;
}

View File

@ -20,7 +20,9 @@ package org.apache.hadoop.ipc;
import org.apache.commons.logging.*;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
@ -590,6 +592,38 @@ public class TestIPC {
Server.RECEIVED_HTTP_REQ_RESPONSE.getBytes());
}
@Test
public void testConnectionRetriesOnSocketTimeoutExceptions() throws Exception {
Configuration conf = new Configuration();
// set max retries to 0
conf.setInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
0);
assertRetriesOnSocketTimeouts(conf, 1);
// set max retries to 3
conf.setInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
3);
assertRetriesOnSocketTimeouts(conf, 4);
}
private void assertRetriesOnSocketTimeouts(Configuration conf,
int maxTimeoutRetries) throws IOException, InterruptedException {
SocketFactory mockFactory = Mockito.mock(SocketFactory.class);
doThrow(new SocketTimeoutException()).when(mockFactory).createSocket();
Client client = new Client(IntWritable.class, conf, mockFactory);
InetSocketAddress address = new InetSocketAddress("127.0.0.1", 9090);
try {
client.call(new IntWritable(RANDOM.nextInt()), address, null, null, 0,
conf);
fail("Not throwing the SocketTimeoutException");
} catch (SocketTimeoutException e) {
Mockito.verify(mockFactory, Mockito.times(maxTimeoutRetries))
.createSocket();
}
}
private void doIpcVersionTest(
byte[] requestData,
byte[] expectedResponse) throws Exception {

View File

@ -0,0 +1,252 @@
Changes for HDFS-1623 branch.
This change list will be merged into the trunk CHANGES.txt when the HDFS-1623
branch is merged.
------------------------------
HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)
HDFS-1974. Introduce active and standy states to the namenode. (suresh)
HDFS-2407. getServerDefaults and getStats don't check operation category (atm)
HDFS-1973. HA: HDFS clients must handle namenode failover and switch over to the new active namenode. (atm)
HDFS-2301. Start/stop appropriate namenode services when transition to active and standby states. (suresh)
HDFS-2231. Configuration changes for HA namenode. (suresh)
HDFS-2418. Change ConfiguredFailoverProxyProvider to take advantage of HDFS-2231. (atm)
HDFS-2393. Mark appropriate methods of ClientProtocol with the idempotent annotation. (atm)
HDFS-2523. Small NN fixes to include HAServiceProtocol and prevent NPE on shutdown. (todd)
HDFS-2577. NN fails to start since it tries to start secret manager in safemode. (todd)
HDFS-2582. Scope dfs.ha.namenodes config by nameservice (todd)
HDFS-2591. MiniDFSCluster support to mix and match federation with HA (todd)
HDFS-1975. Support for sharing the namenode state from active to standby. (jitendra, atm, todd)
HDFS-1971. Send block report from datanode to both active and standby namenodes. (sanjay, todd via suresh)
HDFS-2616. Change DatanodeProtocol#sendHeartbeat() to return HeartbeatResponse. (suresh)
HDFS-2622. Fix TestDFSUpgrade in HA branch. (todd)
HDFS-2612. Handle refreshNameNodes in federated HA clusters (todd)
HDFS-2623. Add test case for hot standby capability (todd)
HDFS-2626. BPOfferService.verifyAndSetNamespaceInfo needs to be synchronized (todd)
HDFS-2624. ConfiguredFailoverProxyProvider doesn't correctly stop ProtocolTranslators (todd)
HDFS-2625. TestDfsOverAvroRpc failing after introduction of HeartbeatResponse type (todd)
HDFS-2627. Determine DN's view of which NN is active based on heartbeat responses (todd)
HDFS-2634. Standby needs to ingest latest edit logs before transitioning to active (todd)
HDFS-2671. NN should throw StandbyException in response to RPCs in STANDBY state (todd)
HDFS-2680. DFSClient should construct failover proxy with exponential backoff (todd)
HDFS-2683. Authority-based lookup of proxy provider fails if path becomes canonicalized (todd)
HDFS-2689. HA: BookKeeperEditLogInputStream doesn't implement isInProgress() (atm)
HDFS-2602. NN should log newly-allocated blocks without losing BlockInfo (atm)
HDFS-2667. Fix transition from active to standby (todd)
HDFS-2684. Fix up some failing unit tests on HA branch (todd)
HDFS-2679. Add interface to query current state to HAServiceProtocol (eli via todd)
HDFS-2677. Web UI should indicate the NN state. (eli via todd)
HDFS-2678. When a FailoverProxyProvider is used, DFSClient should not retry connection ten times before failing over (atm via todd)
HDFS-2682. When a FailoverProxyProvider is used, Client should not retry for 45 times if it is timing out to connect to server. (Uma Maheswara Rao G via todd)
HDFS-2693. Fix synchronization issues around state transition (todd)
HDFS-1972. Fencing mechanism for block invalidations and replications (todd)
HDFS-2714. Fix test cases which use standalone FSNamesystems (todd)
HDFS-2692. Fix bugs related to failover from/into safe mode. (todd)
HDFS-2716. Configuration needs to allow different dfs.http.addresses for each HA NN (todd)
HDFS-2720. Fix MiniDFSCluster HA support to work properly on Windows. (Uma Maheswara Rao G via todd)
HDFS-2291. Allow the StandbyNode to make checkpoints in an HA setup. (todd)
HDFS-2709. Appropriately handle error conditions in EditLogTailer (atm via todd)
HDFS-2730. Refactor shared HA-related test code into HATestUtil class (todd)
HDFS-2762. Fix TestCheckpoint timing out on HA branch. (Uma Maheswara Rao G via todd)
HDFS-2724. NN web UI can throw NPE after startup, before standby state is entered. (todd)
HDFS-2753. Fix standby getting stuck in safemode when blocks are written while SBN is down. (Hari Mankude and todd via todd)
HDFS-2773. Reading edit logs from an earlier version should not leave blocks in under-construction state. (todd)
HDFS-2775. Fix TestStandbyCheckpoints.testBothNodesInStandbyState failing intermittently. (todd)
HDFS-2766. Test for case where standby partially reads log and then performs checkpoint. (atm)
HDFS-2738. FSEditLog.selectinputStreams is reading through in-progress streams even when non-in-progress are requested. (atm)
HDFS-2789. TestHAAdmin.testFailover is failing (eli)
HDFS-2747. Entering safe mode after starting SBN can NPE. (Uma Maheswara Rao G via todd)
HDFS-2772. On transition to active, standby should not swallow ELIE. (atm)
HDFS-2767. ConfiguredFailoverProxyProvider should support NameNodeProtocol. (Uma Maheswara Rao G via todd)
HDFS-2795. Standby NN takes a long time to recover from a dead DN starting up. (todd)
HDFS-2592. Balancer support for HA namenodes. (Uma Maheswara Rao G via todd)
HDFS-2367. Enable the configuration of multiple HA cluster addresses. (atm)
HDFS-2812. When becoming active, the NN should treat all leases as freshly renewed. (todd)
HDFS-2737. Automatically trigger log rolls periodically on the active NN. (todd and atm)
HDFS-2820. Add a simple sanity check for HA config (todd)
HDFS-2688. Add tests for quota tracking in an HA cluster. (todd)
HDFS-2804. Should not mark blocks under-replicated when exiting safemode (todd)
HDFS-2807. Service level authorizartion for HAServiceProtocol. (jitendra)
HDFS-2809. Add test to verify that delegation tokens are honored after failover. (jitendra and atm)
HDFS-2838. NPE in FSNamesystem when in safe mode. (Gregory Chanan via eli)
HDFS-2805. Add a test for a federated cluster with HA NNs. (Brandon Li via jitendra)
HDFS-2841. HAAdmin does not work if security is enabled. (atm)
HDFS-2691. Fixes for pipeline recovery in an HA cluster: report RBW replicas immediately upon pipeline creation. (todd)
HDFS-2824. Fix failover when prior NN died just after creating an edit log segment. (atm via todd)
HDFS-2853. HA: NN fails to start if the shared edits dir is marked required (atm via eli)
HDFS-2845. SBN should not allow browsing of the file system via web UI. (Bikas Saha via atm)
HDFS-2742. HA: observed dataloss in replication stress test. (todd via eli)
HDFS-2870. Fix log level for block debug info in processMisReplicatedBlocks (todd)
HDFS-2859. LOCAL_ADDRESS_MATCHER.match has NPE when called from DFSUtil.getSuffixIDs when the host is incorrect (Bikas Saha via todd)
HDFS-2861. checkpointing should verify that the dfs.http.address has been configured to a non-loopback for peer NN (todd)
HDFS-2860. TestDFSRollback#testRollback is failing. (atm)
HDFS-2769. HA: When HA is enabled with a shared edits dir, that dir should be
marked required. (atm via eli)
HDFS-2863. Failures observed if dfs.edits.dir and shared.edits.dir have same directories. (Bikas Saha via atm)
HDFS-2874. Edit log should log to shared dirs before local dirs. (todd)
HDFS-2890. DFSUtil#getSuffixIDs should skip unset configurations. (atm)
HDFS-2792. Make fsck work. (atm)
HDFS-2808. HA: haadmin should use namenode ids. (eli)
HDFS-2819. Document new HA-related configs in hdfs-default.xml. (eli)
HDFS-2752. HA: exit if multiple shared dirs are configured. (eli)
HDFS-2894. HA: automatically determine the nameservice Id if only one nameservice is configured. (eli)
HDFS-2733. Document HA configuration and CLI. (atm)
HDFS-2794. Active NN may purge edit log files before standby NN has a chance to read them (todd)
HDFS-2901. Improvements for SBN web UI - not show under-replicated/missing blocks. (Brandon Li via jitendra)
HDFS-2905. HA: Standby NN NPE when shared edits dir is deleted. (Bikas Saha via jitendra)
HDFS-2579. Starting delegation token manager during safemode fails. (todd)
HDFS-2510. Add HA-related metrics. (atm)
HDFS-2924. Standby checkpointing fails to authenticate in secure cluster. (todd)
HDFS-2915. HA: TestFailureOfSharedDir.testFailureOfSharedDir() has race condition. (Bikas Saha via jitendra)
HDFS-2912. Namenode not shutting down when shared edits dir is inaccessible. (Bikas Saha via atm)
HDFS-2917. HA: haadmin should not work if run by regular user (eli)
HDFS-2939. TestHAStateTransitions fails on Windows. (Uma Maheswara Rao G via atm)
HDFS-2947. On startup NN throws an NPE in the metrics system. (atm)
HDFS-2942. TestActiveStandbyElectorRealZK fails if build dir does not exist. (atm)
HDFS-2948. NN throws NPE during shutdown if it fails to startup (todd)
HDFS-2909. HA: Inaccessible shared edits dir not getting removed from FSImage storage dirs upon error. (Bikas Saha via jitendra)
HDFS-2934. Allow configs to be scoped to all NNs in the nameservice. (todd)
HDFS-2935. Shared edits dir property should be suffixed with nameservice and namenodeID (todd)
HDFS-2928. ConfiguredFailoverProxyProvider should not create a NameNode proxy with an underlying retry proxy. (Uma Maheswara Rao G via atm)
HDFS-2955. IllegalStateException during standby startup in getCurSegmentTxId. (Hari Mankude via atm)
HDFS-2937. TestDFSHAAdmin needs tests with MiniDFSCluster. (Brandon Li via suresh)
HDFS-2586. Add protobuf service and implementation for HAServiceProtocol. (suresh via atm)
HDFS-2952. NN should not start with upgrade option or with a pending an unfinalized upgrade. (atm)
HDFS-2974. MiniDFSCluster does not delete standby NN name dirs during format. (atm)
HDFS-2929. Stress test and fixes for block synchronization (todd)
HDFS-2972. Small optimization building incremental block report (todd)
HDFS-2973. Re-enable NO_ACK optimization for block deletion. (todd)
HDFS-2922. HA: close out operation categories (eli)
HDFS-2993. HA: BackupNode#checkOperation should permit CHECKPOINT operations (eli)
HDFS-2904. Client support for getting delegation tokens. (todd)
HDFS-3013. HA: NameNode format doesn't pick up dfs.namenode.name.dir.NameServiceId configuration (Mingjie Lai via todd)
HDFS-3019. Fix silent failure of TestEditLogJournalFailures (todd)
HDFS-2958. Sweep for remaining proxy construction which doesn't go through failover path. (atm)
HDFS-2920. fix remaining TODO items. (atm and todd)
HDFS-3027. Implement a simple NN health check. (atm)
HDFS-3023. Optimize entries in edits log for persistBlocks call. (todd)
HDFS-2979. Balancer should use logical uri for creating failover proxy with HA enabled. (atm)
HDFS-3035. Fix failure of TestFileAppendRestart due to OP_UPDATE_BLOCKS (todd)
HDFS-3039. Address findbugs and javadoc warnings on branch. (todd via atm)

View File

@ -256,4 +256,12 @@
<Field name="metrics" />
<Bug pattern="IS2_INCONSISTENT_SYNC" />
</Match>
<!--
This method isn't performance-critical and is much clearer to write as it's written.
-->
<Match>
<Class name="org.apache.hadoop.hdfs.server.datanode.BlockPoolManager" />
<Method name="doRefreshNamenodes" />
<Bug category="PERFORMANCE" />
</Match>
</FindBugsFilter>

View File

@ -387,6 +387,7 @@
<configuration>
<excludes>
<exclude>CHANGES.txt</exclude>
<exclude>CHANGES.HDFS-1623.txt</exclude>
<exclude>.idea/**</exclude>
<exclude>src/main/conf/*</exclude>
<exclude>src/main/docs/**</exclude>

View File

@ -129,6 +129,12 @@ class BookKeeperEditLogInputStream extends EditLogInputStream {
return null;
}
// TODO(HA): Test this.
@Override
public boolean isInProgress() {
return true;
}
/**
* Input stream implementation which can be used by
* FSEditLogOp.Reader

View File

@ -312,8 +312,10 @@ public class BookKeeperJournalManager implements JournalManager {
}
}
// TODO(HA): Handle inProgressOk
@Override
public EditLogInputStream getInputStream(long fromTxnId) throws IOException {
public EditLogInputStream getInputStream(long fromTxnId, boolean inProgressOk)
throws IOException {
for (EditLogLedgerMetadata l : getLedgerList()) {
if (l.getFirstTxId() == fromTxnId) {
try {
@ -329,8 +331,10 @@ public class BookKeeperJournalManager implements JournalManager {
throw new IOException("No ledger for fromTxnId " + fromTxnId + " found.");
}
// TODO(HA): Handle inProgressOk
@Override
public long getNumberOfTransactions(long fromTxnId) throws IOException {
public long getNumberOfTransactions(long fromTxnId, boolean inProgressOk)
throws IOException {
long count = 0;
long expectedStart = 0;
for (EditLogLedgerMetadata l : getLedgerList()) {

View File

@ -195,7 +195,7 @@ public class TestBookKeeperJournalManager {
out.close();
bkjm.finalizeLogSegment(1, 100);
long numTrans = bkjm.getNumberOfTransactions(1);
long numTrans = bkjm.getNumberOfTransactions(1, true);
assertEquals(100, numTrans);
}
@ -218,17 +218,17 @@ public class TestBookKeeperJournalManager {
}
zkc.delete(bkjm.finalizedLedgerZNode(DEFAULT_SEGMENT_SIZE+1, DEFAULT_SEGMENT_SIZE*2), -1);
long numTrans = bkjm.getNumberOfTransactions(1);
long numTrans = bkjm.getNumberOfTransactions(1, true);
assertEquals(DEFAULT_SEGMENT_SIZE, numTrans);
try {
numTrans = bkjm.getNumberOfTransactions(DEFAULT_SEGMENT_SIZE+1);
numTrans = bkjm.getNumberOfTransactions(DEFAULT_SEGMENT_SIZE+1, true);
fail("Should have thrown corruption exception by this point");
} catch (JournalManager.CorruptionException ce) {
// if we get here, everything is going good
}
numTrans = bkjm.getNumberOfTransactions((DEFAULT_SEGMENT_SIZE*2)+1);
numTrans = bkjm.getNumberOfTransactions((DEFAULT_SEGMENT_SIZE*2)+1, true);
assertEquals(DEFAULT_SEGMENT_SIZE, numTrans);
}
@ -262,7 +262,7 @@ public class TestBookKeeperJournalManager {
out.abort();
out.close();
long numTrans = bkjm.getNumberOfTransactions(1);
long numTrans = bkjm.getNumberOfTransactions(1, true);
assertEquals((txid-1), numTrans);
}
@ -357,7 +357,7 @@ public class TestBookKeeperJournalManager {
bkjm.finalizeLogSegment(1, numTransactions);
EditLogInputStream in = bkjm.getInputStream(1);
EditLogInputStream in = bkjm.getInputStream(1, true);
try {
assertEquals(numTransactions,
FSEditLogTestUtil.countTransactionsInStream(in));
@ -392,4 +392,4 @@ public class TestBookKeeperJournalManager {
assertNotNull(zkc.exists(bkjm.finalizedLedgerZNode(1, 100), false));
assertNull(zkc.exists(bkjm.inprogressZNode(), false));
}
}
}

View File

@ -32,6 +32,7 @@ function print_usage(){
echo " namenode run the DFS namenode"
echo " datanode run a DFS datanode"
echo " dfsadmin run a DFS admin client"
echo " haadmin run a DFS HA admin client"
echo " fsck run a DFS filesystem checking utility"
echo " balancer run a cluster balancing utility"
echo " jmxget get JMX exported values from NameNode or DataNode."
@ -86,6 +87,10 @@ elif [ "$COMMAND" = "dfs" ] ; then
elif [ "$COMMAND" = "dfsadmin" ] ; then
CLASS=org.apache.hadoop.hdfs.tools.DFSAdmin
HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
elif [ "$COMMAND" = "haadmin" ] ; then
CLASS=org.apache.hadoop.hdfs.tools.DFSHAAdmin
CLASSPATH=${CLASSPATH}:${TOOL_PATH}
HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"
elif [ "$COMMAND" = "fsck" ] ; then
CLASS=org.apache.hadoop.hdfs.tools.DFSck
HADOOP_OPTS="$HADOOP_OPTS $HADOOP_CLIENT_OPTS"

View File

@ -80,8 +80,7 @@ public class Hdfs extends AbstractFileSystem {
throw new IOException("Incomplete HDFS URI, no host: " + theUri);
}
InetSocketAddress namenode = NameNode.getAddress(theUri.getAuthority());
this.dfs = new DFSClient(namenode, conf, getStatistics());
this.dfs = new DFSClient(theUri, conf, getStatistics());
}
@Override

View File

@ -1,4 +1,3 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -30,6 +29,8 @@ import java.net.InetSocketAddress;
import java.net.NetworkInterface;
import java.net.Socket;
import java.net.SocketException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
@ -60,6 +61,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.permission.FsPermission;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
@ -83,6 +85,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
@ -105,7 +108,8 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenRenewer;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import com.google.common.base.Preconditions;
/********************************************************
* DFSClient can connect to a Hadoop Filesystem and
@ -124,7 +128,9 @@ public class DFSClient implements java.io.Closeable {
public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
final ClientProtocol namenode;
private final InetSocketAddress nnAddress;
/* The service used for delegation tokens */
private Text dtService;
final UserGroupInformation ugi;
volatile boolean clientRunning = true;
private volatile FsServerDefaults serverDefaults;
@ -143,6 +149,9 @@ public class DFSClient implements java.io.Closeable {
* DFSClient configuration
*/
static class Conf {
final int maxFailoverAttempts;
final int failoverSleepBaseMillis;
final int failoverSleepMaxMillis;
final int maxBlockAcquireFailures;
final int confTime;
final int ioBufferSize;
@ -164,6 +173,16 @@ public class DFSClient implements java.io.Closeable {
final boolean useLegacyBlockReader;
Conf(Configuration conf) {
maxFailoverAttempts = conf.getInt(
DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY,
DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT);
failoverSleepBaseMillis = conf.getInt(
DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY,
DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT);
failoverSleepMaxMillis = conf.getInt(
DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY,
DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT);
maxBlockAcquireFailures = conf.getInt(
DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT);
@ -236,6 +255,7 @@ public class DFSClient implements java.io.Closeable {
*/
private final Map<String, DFSOutputStream> filesBeingWritten
= new HashMap<String, DFSOutputStream>();
private boolean shortCircuitLocalReads;
/**
@ -247,59 +267,69 @@ public class DFSClient implements java.io.Closeable {
public DFSClient(Configuration conf) throws IOException {
this(NameNode.getAddress(conf), conf);
}
public DFSClient(InetSocketAddress address, Configuration conf) throws IOException {
this(NameNode.getUri(address), conf);
}
/**
* Same as this(nameNodeAddr, conf, null);
* Same as this(nameNodeUri, conf, null);
* @see #DFSClient(InetSocketAddress, Configuration, org.apache.hadoop.fs.FileSystem.Statistics)
*/
public DFSClient(InetSocketAddress nameNodeAddr, Configuration conf
public DFSClient(URI nameNodeUri, Configuration conf
) throws IOException {
this(nameNodeAddr, conf, null);
this(nameNodeUri, conf, null);
}
/**
* Same as this(nameNodeAddr, null, conf, stats);
* Same as this(nameNodeUri, null, conf, stats);
* @see #DFSClient(InetSocketAddress, ClientProtocol, Configuration, org.apache.hadoop.fs.FileSystem.Statistics)
*/
public DFSClient(InetSocketAddress nameNodeAddr, Configuration conf,
public DFSClient(URI nameNodeUri, Configuration conf,
FileSystem.Statistics stats)
throws IOException {
this(nameNodeAddr, null, conf, stats);
this(nameNodeUri, null, conf, stats);
}
/**
* Create a new DFSClient connected to the given nameNodeAddr or rpcNamenode.
* Exactly one of nameNodeAddr or rpcNamenode must be null.
* Create a new DFSClient connected to the given nameNodeUri or rpcNamenode.
* Exactly one of nameNodeUri or rpcNamenode must be null.
*/
DFSClient(InetSocketAddress nameNodeAddr, ClientProtocol rpcNamenode,
DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
Configuration conf, FileSystem.Statistics stats)
throws IOException {
// Copy only the required DFSClient configuration
this.dfsClientConf = new Conf(conf);
this.conf = conf;
this.stats = stats;
this.nnAddress = nameNodeAddr;
this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
// The hdfsTimeout is currently the same as the ipc timeout
this.hdfsTimeout = Client.getTimeout(conf);
this.ugi = UserGroupInformation.getCurrentUser();
final String authority = nameNodeAddr == null? "null":
nameNodeAddr.getHostName() + ":" + nameNodeAddr.getPort();
final String authority = nameNodeUri == null? "null": nameNodeUri.getAuthority();
this.leaserenewer = LeaseRenewer.getInstance(authority, ugi, this);
this.clientName = leaserenewer.getClientName(dfsClientConf.taskId);
this.socketCache = new SocketCache(dfsClientConf.socketCacheCapacity);
if (nameNodeAddr != null && rpcNamenode == null) {
this.namenode = DFSUtil.createNamenode(nameNodeAddr, conf, ugi);
} else if (nameNodeAddr == null && rpcNamenode != null) {
//This case is used for testing.
if (rpcNamenode != null) {
// This case is used for testing.
Preconditions.checkArgument(nameNodeUri == null);
this.namenode = rpcNamenode;
dtService = null;
} else {
throw new IllegalArgumentException(
"Expecting exactly one of nameNodeAddr and rpcNamenode being null: "
+ "nameNodeAddr=" + nameNodeAddr + ", rpcNamenode=" + rpcNamenode);
Preconditions.checkArgument(nameNodeUri != null,
"null URI");
NameNodeProxies.ProxyAndInfo<ClientProtocol> proxyInfo =
NameNodeProxies.createProxy(conf, nameNodeUri, ClientProtocol.class);
this.dtService = proxyInfo.getDelegationTokenService();
this.namenode = proxyInfo.getProxy();
}
// read directly from the block file if configured.
this.shortCircuitLocalReads = conf.getBoolean(
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
@ -388,20 +418,8 @@ public class DFSClient implements java.io.Closeable {
/**
* Close connections the Namenode.
* The namenode variable is either a rpcProxy passed by a test or
* created using the protocolTranslator which is closeable.
* If closeable then call close, else close using RPC.stopProxy().
*/
void closeConnectionToNamenode() {
if (namenode instanceof Closeable) {
try {
((Closeable) namenode).close();
return;
} catch (IOException e) {
// fall through - lets try the stopProxy
LOG.warn("Exception closing namenode, stopping the proxy");
}
}
RPC.stopProxy(namenode);
}
@ -491,11 +509,13 @@ public class DFSClient implements java.io.Closeable {
*/
public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
throws IOException {
Token<DelegationTokenIdentifier> result =
assert dtService != null;
Token<DelegationTokenIdentifier> token =
namenode.getDelegationToken(renewer);
SecurityUtil.setTokenService(result, nnAddress);
LOG.info("Created " + DelegationTokenIdentifier.stringifyToken(result));
return result;
token.setService(this.dtService);
LOG.info("Created " + DelegationTokenIdentifier.stringifyToken(token));
return token;
}
/**
@ -625,13 +645,8 @@ public class DFSClient implements java.io.Closeable {
@Override
public long renew(Token<?> token, Configuration conf) throws IOException {
Token<DelegationTokenIdentifier> delToken =
(Token<DelegationTokenIdentifier>) token;
LOG.info("Renewing " +
DelegationTokenIdentifier.stringifyToken(delToken));
ClientProtocol nn =
DFSUtil.createNamenode
(SecurityUtil.getTokenServiceAddr(delToken),
conf, UserGroupInformation.getCurrentUser());
(Token<DelegationTokenIdentifier>) token;
ClientProtocol nn = getNNProxy(delToken, conf);
try {
return nn.renewDelegationToken(delToken);
} catch (RemoteException re) {
@ -647,9 +662,7 @@ public class DFSClient implements java.io.Closeable {
(Token<DelegationTokenIdentifier>) token;
LOG.info("Cancelling " +
DelegationTokenIdentifier.stringifyToken(delToken));
ClientProtocol nn = DFSUtil.createNamenode(
SecurityUtil.getTokenServiceAddr(delToken), conf,
UserGroupInformation.getCurrentUser());
ClientProtocol nn = getNNProxy(delToken, conf);
try {
nn.cancelDelegationToken(delToken);
} catch (RemoteException re) {
@ -657,6 +670,31 @@ public class DFSClient implements java.io.Closeable {
AccessControlException.class);
}
}
private static ClientProtocol getNNProxy(
Token<DelegationTokenIdentifier> token, Configuration conf)
throws IOException {
URI uri = HAUtil.getServiceUriFromToken(token);
if (HAUtil.isTokenForLogicalUri(token) &&
!HAUtil.isLogicalUri(conf, uri)) {
// If the token is for a logical nameservice, but the configuration
// we have disagrees about that, we can't actually renew it.
// This can be the case in MR, for example, if the RM doesn't
// have all of the HA clusters configured in its configuration.
throw new IOException("Unable to map logical nameservice URI '" +
uri + "' to a NameNode. Local configuration does not have " +
"a failover proxy provider configured.");
}
NameNodeProxies.ProxyAndInfo<ClientProtocol> info =
NameNodeProxies.createProxy(conf, uri, ClientProtocol.class);
assert info.getDelegationTokenService().equals(token.getService()) :
"Returned service '" + info.getDelegationTokenService().toString() +
"' doesn't match expected service '" +
token.getService().toString() + "'";
return info.getProxy();
}
@Override
public boolean isManaged(Token<?> token) throws IOException {

View File

@ -48,6 +48,19 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_DEFAULT = "DEFAULT";
public static final String DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY = "dfs.client.socketcache.capacity";
public static final int DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT = 16;
// HA related configuration
public static final String DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX = "dfs.client.failover.proxy.provider";
public static final String DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY = "dfs.client.failover.max.attempts";
public static final int DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT = 15;
public static final String DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY = "dfs.client.failover.sleep.base.millis";
public static final int DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT = 500;
public static final String DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY = "dfs.client.failover.sleep.max.millis";
public static final int DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT = 15000;
public static final String DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_KEY = "dfs.client.failover.connection.retries";
public static final int DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_DEFAULT = 0;
public static final String DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY = "dfs.client.failover.connection.retries.on.timeouts";
public static final int DFS_CLIENT_FAILOVER_CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT = 0;
public static final String DFS_NAMENODE_BACKUP_ADDRESS_KEY = "dfs.namenode.backup.address";
public static final String DFS_NAMENODE_BACKUP_ADDRESS_DEFAULT = "localhost:50100";
@ -120,6 +133,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final boolean DFS_WEBHDFS_ENABLED_DEFAULT = false;
public static final String DFS_PERMISSIONS_ENABLED_KEY = "dfs.permissions.enabled";
public static final boolean DFS_PERMISSIONS_ENABLED_DEFAULT = true;
public static final String DFS_PERSIST_BLOCKS_KEY = "dfs.persist.blocks";
public static final boolean DFS_PERSIST_BLOCKS_DEFAULT = false;
public static final String DFS_PERMISSIONS_SUPERUSERGROUP_KEY = "dfs.permissions.superusergroup";
public static final String DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT = "supergroup";
public static final String DFS_ADMIN = "dfs.cluster.administrators";
@ -131,6 +146,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final boolean DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_DEFAULT = true;
public static final String DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_KEY = "dfs.namenode.num.checkpoints.retained";
public static final int DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_DEFAULT = 2;
public static final String DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY = "dfs.namenode.num.extra.edits.retained";
public static final int DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_DEFAULT = 1000000; //1M
public static final String DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY = "dfs.namenode.edits.dir.minimum";
public static final int DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT = 1;
@ -150,6 +168,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final long DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT = 24*60*60*1000; // 1 day
public static final String DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY = "dfs.namenode.delegation.token.max-lifetime";
public static final long DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT = 7*24*60*60*1000; // 7 days
public static final String DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY = "dfs.namenode.delegation.token.always-use"; // for tests
public static final boolean DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT = false;
//Filesystem limit keys
public static final String DFS_NAMENODE_MAX_COMPONENT_LENGTH_KEY = "dfs.namenode.fs-limits.max-component-length";
@ -165,6 +185,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_NAMENODE_HTTPS_PORT_DEFAULT;
public static final String DFS_NAMENODE_NAME_DIR_KEY = "dfs.namenode.name.dir";
public static final String DFS_NAMENODE_EDITS_DIR_KEY = "dfs.namenode.edits.dir";
public static final String DFS_NAMENODE_SHARED_EDITS_DIR_KEY = "dfs.namenode.shared.edits.dir";
public static final String DFS_NAMENODE_EDITS_PLUGIN_PREFIX = "dfs.namenode.edits.journal-plugin";
public static final String DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY = "dfs.namenode.edits.dir.required";
public static final String DFS_CLIENT_READ_PREFETCH_SIZE_KEY = "dfs.client.read.prefetch.size";
@ -298,8 +319,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_NAMENODE_NAME_CACHE_THRESHOLD_KEY = "dfs.namenode.name.cache.threshold";
public static final int DFS_NAMENODE_NAME_CACHE_THRESHOLD_DEFAULT = 10;
public static final String DFS_FEDERATION_NAMESERVICES = "dfs.federation.nameservices";
public static final String DFS_FEDERATION_NAMESERVICE_ID = "dfs.federation.nameservice.id";
public static final String DFS_FEDERATION_NAMESERVICES = "dfs.federation.nameservices";
public static final String DFS_FEDERATION_NAMESERVICE_ID = "dfs.federation.nameservice.id";
public static final String DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY = "dfs.namenode.resource.check.interval";
public static final int DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_DEFAULT = 5000;
public static final String DFS_NAMENODE_DU_RESERVED_KEY = "dfs.namenode.resource.du.reserved";
@ -309,5 +330,16 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final int DFS_NAMENODE_CHECKED_VOLUMES_MINIMUM_DEFAULT = 1;
public static final String DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY = "dfs.web.authentication.kerberos.principal";
public static final String DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY = "dfs.web.authentication.kerberos.keytab";
public static final String DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY = "dfs.block.local-path-access.user";
// HA related configuration
public static final String DFS_HA_NAMENODES_KEY_PREFIX = "dfs.ha.namenodes";
public static final String DFS_HA_NAMENODE_ID_KEY = "dfs.ha.namenode.id";
public static final String DFS_HA_STANDBY_CHECKPOINTS_KEY = "dfs.ha.standby.checkpoints";
public static final boolean DFS_HA_STANDBY_CHECKPOINTS_DEFAULT = true;
public static final String DFS_HA_LOGROLL_PERIOD_KEY = "dfs.ha.log-roll.period";
public static final int DFS_HA_LOGROLL_PERIOD_DEFAULT = 2 * 60; // 2m
public static final String DFS_HA_TAILEDITS_PERIOD_KEY = "dfs.ha.tail-edits.period";
public static final int DFS_HA_TAILEDITS_PERIOD_DEFAULT = 60; // 1m
}

View File

@ -18,24 +18,21 @@
package org.apache.hadoop.hdfs;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_FEDERATION_NAMESERVICES;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_FEDERATION_NAMESERVICE_ID;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.net.InetSocketAddress;
import java.net.URI;
import java.net.URISyntaxException;
import java.security.SecureRandom;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.StringTokenizer;
import javax.net.SocketFactory;
@ -46,9 +43,9 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolTranslatorPB;
@ -59,11 +56,19 @@ import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.NodeBase;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import com.google.common.base.Joiner;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.protobuf.BlockingService;
@InterfaceAudience.Private
public class DFSUtil {
public static final Log LOG = LogFactory.getLog(DFSUtil.class.getName());
private DFSUtil() { /* Hidden constructor */ }
private static final ThreadLocal<Random> RANDOM = new ThreadLocal<Random>() {
@Override
protected Random initialValue() {
@ -101,13 +106,20 @@ public class DFSUtil {
a.isDecommissioned() ? 1 : -1;
}
};
/**
* Address matcher for matching an address to local address
*/
static final AddressMatcher LOCAL_ADDRESS_MATCHER = new AddressMatcher() {
public boolean match(InetSocketAddress s) {
return NetUtils.isLocalAddress(s.getAddress());
};
};
/**
* Whether the pathname is valid. Currently prohibits relative paths,
* and names which contain a ":" or "/"
*/
public static boolean isValidName(String src) {
// Path must be absolute.
if (!src.startsWith(Path.SEPARATOR)) {
return false;
@ -304,12 +316,38 @@ public class DFSUtil {
/**
* Returns collection of nameservice Ids from the configuration.
* @param conf configuration
* @return collection of nameservice Ids
* @return collection of nameservice Ids, or null if not specified
*/
public static Collection<String> getNameServiceIds(Configuration conf) {
return conf.getStringCollection(DFS_FEDERATION_NAMESERVICES);
return conf.getTrimmedStringCollection(DFS_FEDERATION_NAMESERVICES);
}
/**
* @return <code>coll</code> if it is non-null and non-empty. Otherwise,
* returns a list with a single null value.
*/
private static Collection<String> emptyAsSingletonNull(Collection<String> coll) {
if (coll == null || coll.isEmpty()) {
return Collections.singletonList(null);
} else {
return coll;
}
}
/**
* Namenode HighAvailability related configuration.
* Returns collection of namenode Ids from the configuration. One logical id
* for each namenode in the in the HA setup.
*
* @param conf configuration
* @param nsId the nameservice ID to look at, or null for non-federated
* @return collection of namenode Ids
*/
public static Collection<String> getNameNodeIds(Configuration conf, String nsId) {
String key = addSuffix(DFS_HA_NAMENODES_KEY_PREFIX, nsId);
return conf.getTrimmedStringCollection(key);
}
/**
* Given a list of keys in the order of preference, returns a value
* for the key in the given order from the configuration.
@ -323,9 +361,7 @@ public class DFSUtil {
Configuration conf, String... keys) {
String value = null;
for (String key : keys) {
if (keySuffix != null) {
key += "." + keySuffix;
}
key = addSuffix(key, keySuffix);
value = conf.get(key);
if (value != null) {
break;
@ -337,36 +373,84 @@ public class DFSUtil {
return value;
}
/** Add non empty and non null suffix to a key */
private static String addSuffix(String key, String suffix) {
if (suffix == null || suffix.isEmpty()) {
return key;
}
assert !suffix.startsWith(".") :
"suffix '" + suffix + "' should not already have '.' prepended.";
return key + "." + suffix;
}
/** Concatenate list of suffix strings '.' separated */
private static String concatSuffixes(String... suffixes) {
if (suffixes == null) {
return null;
}
return Joiner.on(".").skipNulls().join(suffixes);
}
/**
* Returns list of InetSocketAddress for a given set of keys.
* @param conf configuration
* @param defaultAddress default address to return in case key is not found
* @param keys Set of keys to look for in the order of preference
* @return list of InetSocketAddress corresponding to the key
* Return configuration key of format key.suffix1.suffix2...suffixN
*/
private static List<InetSocketAddress> getAddresses(Configuration conf,
public static String addKeySuffixes(String key, String... suffixes) {
String keySuffix = concatSuffixes(suffixes);
return addSuffix(key, keySuffix);
}
/**
* Returns the configured address for all NameNodes in the cluster.
* @param conf configuration
* @param defaultAddress default address to return in case key is not found.
* @param keys Set of keys to look for in the order of preference
* @return a map(nameserviceId to map(namenodeId to InetSocketAddress))
*/
private static Map<String, Map<String, InetSocketAddress>>
getAddresses(Configuration conf,
String defaultAddress, String... keys) {
Collection<String> nameserviceIds = getNameServiceIds(conf);
List<InetSocketAddress> isas = new ArrayList<InetSocketAddress>();
// Configuration with a single namenode
if (nameserviceIds == null || nameserviceIds.isEmpty()) {
String address = getConfValue(defaultAddress, null, conf, keys);
if (address == null) {
return null;
}
isas.add(NetUtils.createSocketAddr(address));
} else {
// Get the namenodes for all the configured nameServiceIds
for (String nameserviceId : nameserviceIds) {
String address = getConfValue(null, nameserviceId, conf, keys);
if (address == null) {
return null;
}
isas.add(NetUtils.createSocketAddr(address));
// Look for configurations of the form <key>[.<nameserviceId>][.<namenodeId>]
// across all of the configured nameservices and namenodes.
Map<String, Map<String, InetSocketAddress>> ret = Maps.newHashMap();
for (String nsId : emptyAsSingletonNull(nameserviceIds)) {
Map<String, InetSocketAddress> isas =
getAddressesForNameserviceId(conf, nsId, defaultAddress, keys);
if (!isas.isEmpty()) {
ret.put(nsId, isas);
}
}
return isas;
return ret;
}
private static Map<String, InetSocketAddress> getAddressesForNameserviceId(
Configuration conf, String nsId, String defaultValue,
String[] keys) {
Collection<String> nnIds = getNameNodeIds(conf, nsId);
Map<String, InetSocketAddress> ret = Maps.newHashMap();
for (String nnId : emptyAsSingletonNull(nnIds)) {
String suffix = concatSuffixes(nsId, nnId);
String address = getConfValue(defaultValue, suffix, conf, keys);
if (address != null) {
InetSocketAddress isa = NetUtils.createSocketAddr(address);
ret.put(nnId, isa);
}
}
return ret;
}
/**
* Returns list of InetSocketAddress corresponding to HA NN RPC addresses from
* the configuration.
*
* @param conf configuration
* @return list of InetSocketAddresses
* @throws IOException if no addresses are configured
*/
public static Map<String, Map<String, InetSocketAddress>> getHaNnRpcAddresses(
Configuration conf) {
return getAddresses(conf, null, DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY);
}
/**
@ -377,11 +461,11 @@ public class DFSUtil {
* @return list of InetSocketAddresses
* @throws IOException on error
*/
public static List<InetSocketAddress> getBackupNodeAddresses(
public static Map<String, Map<String, InetSocketAddress>> getBackupNodeAddresses(
Configuration conf) throws IOException {
List<InetSocketAddress> addressList = getAddresses(conf,
Map<String, Map<String, InetSocketAddress>> addressList = getAddresses(conf,
null, DFS_NAMENODE_BACKUP_ADDRESS_KEY);
if (addressList == null) {
if (addressList.isEmpty()) {
throw new IOException("Incorrect configuration: backup node address "
+ DFS_NAMENODE_BACKUP_ADDRESS_KEY + " is not configured.");
}
@ -396,11 +480,11 @@ public class DFSUtil {
* @return list of InetSocketAddresses
* @throws IOException on error
*/
public static List<InetSocketAddress> getSecondaryNameNodeAddresses(
public static Map<String, Map<String, InetSocketAddress>> getSecondaryNameNodeAddresses(
Configuration conf) throws IOException {
List<InetSocketAddress> addressList = getAddresses(conf, null,
Map<String, Map<String, InetSocketAddress>> addressList = getAddresses(conf, null,
DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY);
if (addressList == null) {
if (addressList.isEmpty()) {
throw new IOException("Incorrect configuration: secondary namenode address "
+ DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY + " is not configured.");
}
@ -420,7 +504,7 @@ public class DFSUtil {
* @return list of InetSocketAddress
* @throws IOException on error
*/
public static List<InetSocketAddress> getNNServiceRpcAddresses(
public static Map<String, Map<String, InetSocketAddress>> getNNServiceRpcAddresses(
Configuration conf) throws IOException {
// Use default address as fall back
String defaultAddress;
@ -430,9 +514,10 @@ public class DFSUtil {
defaultAddress = null;
}
List<InetSocketAddress> addressList = getAddresses(conf, defaultAddress,
Map<String, Map<String, InetSocketAddress>> addressList =
getAddresses(conf, defaultAddress,
DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, DFS_NAMENODE_RPC_ADDRESS_KEY);
if (addressList == null) {
if (addressList.isEmpty()) {
throw new IOException("Incorrect configuration: namenode address "
+ DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY + " or "
+ DFS_NAMENODE_RPC_ADDRESS_KEY
@ -442,10 +527,154 @@ public class DFSUtil {
}
/**
* Given the InetSocketAddress for any configured communication with a
* namenode, this method returns the corresponding nameservice ID,
* by doing a reverse lookup on the list of nameservices until it
* finds a match.
* Flatten the given map, as returned by other functions in this class,
* into a flat list of {@link ConfiguredNNAddress} instances.
*/
public static List<ConfiguredNNAddress> flattenAddressMap(
Map<String, Map<String, InetSocketAddress>> map) {
List<ConfiguredNNAddress> ret = Lists.newArrayList();
for (Map.Entry<String, Map<String, InetSocketAddress>> entry :
map.entrySet()) {
String nsId = entry.getKey();
Map<String, InetSocketAddress> nnMap = entry.getValue();
for (Map.Entry<String, InetSocketAddress> e2 : nnMap.entrySet()) {
String nnId = e2.getKey();
InetSocketAddress addr = e2.getValue();
ret.add(new ConfiguredNNAddress(nsId, nnId, addr));
}
}
return ret;
}
/**
* Format the given map, as returned by other functions in this class,
* into a string suitable for debugging display. The format of this string
* should not be considered an interface, and is liable to change.
*/
public static String addressMapToString(
Map<String, Map<String, InetSocketAddress>> map) {
StringBuilder b = new StringBuilder();
for (Map.Entry<String, Map<String, InetSocketAddress>> entry :
map.entrySet()) {
String nsId = entry.getKey();
Map<String, InetSocketAddress> nnMap = entry.getValue();
b.append("Nameservice <").append(nsId).append(">:").append("\n");
for (Map.Entry<String, InetSocketAddress> e2 : nnMap.entrySet()) {
b.append(" NN ID ").append(e2.getKey())
.append(" => ").append(e2.getValue()).append("\n");
}
}
return b.toString();
}
public static String nnAddressesAsString(Configuration conf) {
Map<String, Map<String, InetSocketAddress>> addresses =
getHaNnRpcAddresses(conf);
return addressMapToString(addresses);
}
/**
* Represent one of the NameNodes configured in the cluster.
*/
public static class ConfiguredNNAddress {
private final String nameserviceId;
private final String namenodeId;
private final InetSocketAddress addr;
private ConfiguredNNAddress(String nameserviceId, String namenodeId,
InetSocketAddress addr) {
this.nameserviceId = nameserviceId;
this.namenodeId = namenodeId;
this.addr = addr;
}
public String getNameserviceId() {
return nameserviceId;
}
public String getNamenodeId() {
return namenodeId;
}
public InetSocketAddress getAddress() {
return addr;
}
@Override
public String toString() {
return "ConfiguredNNAddress[nsId=" + nameserviceId + ";" +
"nnId=" + namenodeId + ";addr=" + addr + "]";
}
}
/**
* Get a URI for each configured nameservice. If a nameservice is
* HA-enabled, then the logical URI of the nameservice is returned. If the
* nameservice is not HA-enabled, then a URI corresponding to an RPC address
* of the single NN for that nameservice is returned, preferring the service
* RPC address over the client RPC address.
*
* @param conf configuration
* @return a collection of all configured NN URIs, preferring service
* addresses
*/
public static Collection<URI> getNsServiceRpcUris(Configuration conf) {
return getNameServiceUris(conf,
DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY);
}
/**
* Get a URI for each configured nameservice. If a nameservice is
* HA-enabled, then the logical URI of the nameservice is returned. If the
* nameservice is not HA-enabled, then a URI corresponding to the address of
* the single NN for that nameservice is returned.
*
* @param conf configuration
* @param keys configuration keys to try in order to get the URI for non-HA
* nameservices
* @return a collection of all configured NN URIs
*/
public static Collection<URI> getNameServiceUris(Configuration conf,
String... keys) {
Set<URI> ret = new HashSet<URI>();
for (String nsId : getNameServiceIds(conf)) {
if (HAUtil.isHAEnabled(conf, nsId)) {
// Add the logical URI of the nameservice.
try {
ret.add(new URI(HdfsConstants.HDFS_URI_SCHEME + "://" + nsId));
} catch (URISyntaxException ue) {
throw new IllegalArgumentException(ue);
}
} else {
// Add the URI corresponding to the address of the NN.
for (String key : keys) {
String addr = conf.get(concatSuffixes(key, nsId));
if (addr != null) {
ret.add(createUri(HdfsConstants.HDFS_URI_SCHEME,
NetUtils.createSocketAddr(addr)));
break;
}
}
}
}
// Add the generic configuration keys.
for (String key : keys) {
String addr = conf.get(key);
if (addr != null) {
ret.add(createUri("hdfs", NetUtils.createSocketAddr(addr)));
break;
}
}
return ret;
}
/**
* Given the InetSocketAddress this method returns the nameservice Id
* corresponding to the key with matching address, by doing a reverse
* lookup on the list of nameservices until it finds a match.
*
* Since the process of resolving URIs to Addresses is slightly expensive,
* this utility method should not be used in performance-critical routines.
@ -463,91 +692,109 @@ public class DFSUtil {
* not the NameServiceId-suffixed keys.
* @return nameserviceId, or null if no match found
*/
public static String getNameServiceIdFromAddress(Configuration conf,
InetSocketAddress address, String... keys) {
Collection<String> nameserviceIds = getNameServiceIds(conf);
public static String getNameServiceIdFromAddress(final Configuration conf,
final InetSocketAddress address, String... keys) {
// Configuration with a single namenode and no nameserviceId
if (nameserviceIds == null || nameserviceIds.isEmpty()) {
return null;
}
// Get the candidateAddresses for all the configured nameServiceIds
for (String nameserviceId : nameserviceIds) {
for (String key : keys) {
String candidateAddress = conf.get(
getNameServiceIdKey(key, nameserviceId));
if (candidateAddress != null
&& address.equals(NetUtils.createSocketAddr(candidateAddress)))
return nameserviceId;
}
}
// didn't find a match
return null;
String[] ids = getSuffixIDs(conf, address, keys);
return (ids != null) ? ids[0] : null;
}
/**
* return server http or https address from the configuration
* return server http or https address from the configuration for a
* given namenode rpc address.
* @param conf
* @param namenode - namenode address
* @param namenodeAddr - namenode RPC address
* @param httpsAddress -If true, and if security is enabled, returns server
* https address. If false, returns server http address.
* @return server http or https address
*/
public static String getInfoServer(
InetSocketAddress namenode, Configuration conf, boolean httpsAddress) {
String httpAddress = null;
String httpAddressKey = (UserGroupInformation.isSecurityEnabled()
&& httpsAddress) ? DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY
: DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY;
String httpAddressDefault = (UserGroupInformation.isSecurityEnabled()
&& httpsAddress) ? DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT
: DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_DEFAULT;
if(namenode != null) {
InetSocketAddress namenodeAddr, Configuration conf, boolean httpsAddress) {
boolean securityOn = UserGroupInformation.isSecurityEnabled();
String httpAddressKey = (securityOn && httpsAddress) ?
DFS_NAMENODE_HTTPS_ADDRESS_KEY : DFS_NAMENODE_HTTP_ADDRESS_KEY;
String httpAddressDefault = (securityOn && httpsAddress) ?
DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT : DFS_NAMENODE_HTTP_ADDRESS_DEFAULT;
String suffixes[];
if (namenodeAddr != null) {
// if non-default namenode, try reverse look up
// the nameServiceID if it is available
String nameServiceId = DFSUtil.getNameServiceIdFromAddress(
conf, namenode,
suffixes = getSuffixIDs(conf, namenodeAddr,
DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY);
if (nameServiceId != null) {
httpAddress = conf.get(DFSUtil.getNameServiceIdKey(
httpAddressKey, nameServiceId));
}
}
// else - Use non-federation style configuration
if (httpAddress == null) {
httpAddress = conf.get(httpAddressKey, httpAddressDefault);
} else {
suffixes = new String[2];
}
return httpAddress;
return getSuffixedConf(conf, httpAddressKey, httpAddressDefault, suffixes);
}
/**
* @return key specific to a nameserviceId from a generic key
* Substitute a default host in the case that an address has been configured
* with a wildcard. This is used, for example, when determining the HTTP
* address of the NN -- if it's configured to bind to 0.0.0.0, we want to
* substitute the hostname from the filesystem URI rather than trying to
* connect to 0.0.0.0.
* @param configuredAddress the address found in the configuration
* @param defaultHost the host to substitute with, if configuredAddress
* is a local/wildcard address.
* @return the substituted address
* @throws IOException if it is a wildcard address and security is enabled
*/
public static String getNameServiceIdKey(String key, String nameserviceId) {
return key + "." + nameserviceId;
public static String substituteForWildcardAddress(String configuredAddress,
String defaultHost) throws IOException {
InetSocketAddress sockAddr = NetUtils.createSocketAddr(configuredAddress);
if (sockAddr.getAddress().isAnyLocalAddress()) {
if(UserGroupInformation.isSecurityEnabled()) {
throw new IOException("Cannot use a wildcard address with security. " +
"Must explicitly set bind address for Kerberos");
}
return defaultHost + ":" + sockAddr.getPort();
} else {
return configuredAddress;
}
}
private static String getSuffixedConf(Configuration conf,
String key, String defaultVal, String[] suffixes) {
String ret = conf.get(DFSUtil.addKeySuffixes(key, suffixes));
if (ret != null) {
return ret;
}
return conf.get(key, defaultVal);
}
/**
* Sets the node specific setting into generic configuration key. Looks up
* value of "key.nameserviceId" and if found sets that value into generic key
* in the conf. Note that this only modifies the runtime conf.
* value of "key.nameserviceId.namenodeId" and if found sets that value into
* generic key in the conf. If this is not found, falls back to
* "key.nameserviceId" and then the unmodified key.
*
* Note that this only modifies the runtime conf.
*
* @param conf
* Configuration object to lookup specific key and to set the value
* to the key passed. Note the conf object is modified.
* @param nameserviceId
* nameservice Id to construct the node specific key.
* nameservice Id to construct the node specific key. Pass null if
* federation is not configuration.
* @param nnId
* namenode Id to construct the node specific key. Pass null if
* HA is not configured.
* @param keys
* The key for which node specific value is looked up
*/
public static void setGenericConf(Configuration conf,
String nameserviceId, String... keys) {
String nameserviceId, String nnId, String... keys) {
for (String key : keys) {
String value = conf.get(getNameServiceIdKey(key, nameserviceId));
String value = conf.get(addKeySuffixes(key, nameserviceId, nnId));
if (value != null) {
conf.set(key, value);
continue;
}
value = conf.get(addKeySuffixes(key, nameserviceId));
if (value != null) {
conf.set(key, value);
}
@ -572,34 +819,7 @@ public class DFSUtil {
public static int roundBytesToGB(long bytes) {
return Math.round((float)bytes/ 1024 / 1024 / 1024);
}
/** Create a {@link NameNode} proxy */
public static ClientProtocol createNamenode(Configuration conf)
throws IOException {
return createNamenode(NameNode.getAddress(conf), conf);
}
/** Create a {@link NameNode} proxy */
public static ClientProtocol createNamenode( InetSocketAddress nameNodeAddr,
Configuration conf) throws IOException {
return createNamenode(nameNodeAddr, conf,
UserGroupInformation.getCurrentUser());
}
/** Create a {@link NameNode} proxy */
public static ClientProtocol createNamenode( InetSocketAddress nameNodeAddr,
Configuration conf, UserGroupInformation ugi) throws IOException {
/**
* Currently we have simply burnt-in support for a SINGLE
* protocol - protocolPB. This will be replaced
* by a way to pick the right protocol based on the
* version of the target server.
*/
return new org.apache.hadoop.hdfs.protocolPB.
ClientNamenodeProtocolTranslatorPB(nameNodeAddr, conf, ugi);
}
/** Create a {@link ClientDatanodeProtocol} proxy */
public static ClientDatanodeProtocol createClientDatanodeProtocolProxy(
DatanodeID datanodeid, Configuration conf, int socketTimeout,
@ -622,9 +842,9 @@ public class DFSUtil {
SocketFactory factory) throws IOException {
return new ClientDatanodeProtocolTranslatorPB(addr, ticket, conf, factory);
}
/**
* Get name service Id for the {@link NameNode} based on namenode RPC address
* Get nameservice Id for the {@link NameNode} based on namenode RPC address
* matching the local node address.
*/
public static String getNamenodeNameServiceId(Configuration conf) {
@ -632,7 +852,7 @@ public class DFSUtil {
}
/**
* Get name service Id for the BackupNode based on backup node RPC address
* Get nameservice Id for the BackupNode based on backup node RPC address
* matching the local node address.
*/
public static String getBackupNameServiceId(Configuration conf) {
@ -640,7 +860,7 @@ public class DFSUtil {
}
/**
* Get name service Id for the secondary node based on secondary http address
* Get nameservice Id for the secondary node based on secondary http address
* matching the local node address.
*/
public static String getSecondaryNameServiceId(Configuration conf) {
@ -652,13 +872,14 @@ public class DFSUtil {
* the address of the local node.
*
* If {@link DFSConfigKeys#DFS_FEDERATION_NAMESERVICE_ID} is not specifically
* configured, this method determines the nameservice Id by matching the local
* nodes address with the configured addresses. When a match is found, it
* returns the nameservice Id from the corresponding configuration key.
* configured, and more than one nameservice Id is configured, this method
* determines the nameservice Id by matching the local node's address with the
* configured addresses. When a match is found, it returns the nameservice Id
* from the corresponding configuration key.
*
* @param conf Configuration
* @param addressKey configuration key to get the address.
* @return name service Id on success, null on failure.
* @return nameservice Id on success, null if federation is not configured.
* @throws HadoopIllegalArgumentException on error
*/
private static String getNameServiceId(Configuration conf, String addressKey) {
@ -666,34 +887,106 @@ public class DFSUtil {
if (nameserviceId != null) {
return nameserviceId;
}
Collection<String> ids = getNameServiceIds(conf);
if (ids == null || ids.size() == 0) {
// Not federation configuration, hence no nameservice Id
return null;
Collection<String> nsIds = getNameServiceIds(conf);
if (1 == nsIds.size()) {
return nsIds.toArray(new String[1])[0];
}
String nnId = conf.get(DFS_HA_NAMENODE_ID_KEY);
// Match the rpc address with that of local address
return getSuffixIDs(conf, addressKey, null, nnId, LOCAL_ADDRESS_MATCHER)[0];
}
/**
* Returns nameservice Id and namenode Id when the local host matches the
* configuration parameter {@code addressKey}.<nameservice Id>.<namenode Id>
*
* @param conf Configuration
* @param addressKey configuration key corresponding to the address.
* @param knownNsId only look at configs for the given nameservice, if not-null
* @param knownNNId only look at configs for the given namenode, if not null
* @param matcher matching criteria for matching the address
* @return Array with nameservice Id and namenode Id on success. First element
* in the array is nameservice Id and second element is namenode Id.
* Null value indicates that the configuration does not have the the
* Id.
* @throws HadoopIllegalArgumentException on error
*/
static String[] getSuffixIDs(final Configuration conf, final String addressKey,
String knownNsId, String knownNNId,
final AddressMatcher matcher) {
String nameserviceId = null;
String namenodeId = null;
int found = 0;
for (String id : ids) {
String addr = conf.get(getNameServiceIdKey(addressKey, id));
InetSocketAddress s = NetUtils.createSocketAddr(addr);
if (NetUtils.isLocalAddress(s.getAddress())) {
nameserviceId = id;
found++;
Collection<String> nsIds = getNameServiceIds(conf);
for (String nsId : emptyAsSingletonNull(nsIds)) {
if (knownNsId != null && !knownNsId.equals(nsId)) {
continue;
}
Collection<String> nnIds = getNameNodeIds(conf, nsId);
for (String nnId : emptyAsSingletonNull(nnIds)) {
if (LOG.isTraceEnabled()) {
LOG.trace(String.format("addressKey: %s nsId: %s nnId: %s",
addressKey, nsId, nnId));
}
if (knownNNId != null && !knownNNId.equals(nnId)) {
continue;
}
String key = addKeySuffixes(addressKey, nsId, nnId);
String addr = conf.get(key);
if (addr == null) {
continue;
}
InetSocketAddress s = null;
try {
s = NetUtils.createSocketAddr(addr);
} catch (Exception e) {
LOG.warn("Exception in creating socket address " + addr, e);
continue;
}
if (!s.isUnresolved() && matcher.match(s)) {
nameserviceId = nsId;
namenodeId = nnId;
found++;
}
}
}
if (found > 1) { // Only one address must match the local address
throw new HadoopIllegalArgumentException(
"Configuration has multiple RPC addresses that matches "
+ "the local node's address. Please configure the system with "
+ "the parameter " + DFS_FEDERATION_NAMESERVICE_ID);
String msg = "Configuration has multiple addresses that match "
+ "local node's address. Please configure the system with "
+ DFS_FEDERATION_NAMESERVICE_ID + " and "
+ DFS_HA_NAMENODE_ID_KEY;
throw new HadoopIllegalArgumentException(msg);
}
if (found == 0) {
throw new HadoopIllegalArgumentException("Configuration address "
+ addressKey + " is missing in configuration with name service Id");
return new String[] { nameserviceId, namenodeId };
}
/**
* For given set of {@code keys} adds nameservice Id and or namenode Id
* and returns {nameserviceId, namenodeId} when address match is found.
* @see #getSuffixIDs(Configuration, String, AddressMatcher)
*/
static String[] getSuffixIDs(final Configuration conf,
final InetSocketAddress address, final String... keys) {
AddressMatcher matcher = new AddressMatcher() {
@Override
public boolean match(InetSocketAddress s) {
return address.equals(s);
}
};
for (String key : keys) {
String[] ids = getSuffixIDs(conf, key, null, null, matcher);
if (ids != null && (ids [0] != null || ids[1] != null)) {
return ids;
}
}
return nameserviceId;
return null;
}
private interface AddressMatcher {
public boolean match(InetSocketAddress s);
}
/** Create a URI from the scheme and address */
@ -719,4 +1012,39 @@ public class DFSUtil {
RPC.setProtocolEngine(conf, protocol, ProtobufRpcEngine.class);
server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, protocol, service);
}
/**
* Map a logical namenode ID to its service address. Use the given
* nameservice if specified, or the configured one if none is given.
*
* @param conf Configuration
* @param nsId which nameservice nnId is a part of, optional
* @param nnId the namenode ID to get the service addr for
* @return the service addr, null if it could not be determined
*/
public static String getNamenodeServiceAddr(final Configuration conf,
String nsId, String nnId) {
if (nsId == null) {
Collection<String> nsIds = getNameServiceIds(conf);
if (1 == nsIds.size()) {
nsId = nsIds.toArray(new String[1])[0];
} else {
// No nameservice ID was given and more than one is configured
return null;
}
}
String serviceAddrKey = concatSuffixes(
DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, nsId, nnId);
String addrKey = concatSuffixes(
DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nsId, nnId);
String serviceRpcAddr = conf.get(serviceAddrKey);
if (serviceRpcAddr == null) {
serviceRpcAddr = conf.get(addrKey);
}
return serviceRpcAddr;
}
}

View File

@ -106,8 +106,7 @@ public class DistributedFileSystem extends FileSystem {
throw new IOException("Incomplete HDFS URI, no host: "+ uri);
}
InetSocketAddress namenode = NameNode.getAddress(uri.getAuthority());
this.dfs = new DFSClient(namenode, conf, statistics);
this.dfs = new DFSClient(uri, conf, statistics);
this.uri = URI.create(uri.getScheme()+"://"+uri.getAuthority());
this.workingDir = getHomeDirectory();
}

View File

@ -0,0 +1,261 @@
/**
* 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;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.HA_DT_SERVICE_PREFIX;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
public class HAUtil {
private static final Log LOG =
LogFactory.getLog(HAUtil.class);
private HAUtil() { /* Hidden constructor */ }
/**
* Returns true if HA for namenode is configured for the given nameservice
*
* @param conf Configuration
* @param nsId nameservice, or null if no federated NS is configured
* @return true if HA is configured in the configuration; else false.
*/
public static boolean isHAEnabled(Configuration conf, String nsId) {
Map<String, Map<String, InetSocketAddress>> addresses =
DFSUtil.getHaNnRpcAddresses(conf);
if (addresses == null) return false;
Map<String, InetSocketAddress> nnMap = addresses.get(nsId);
return nnMap != null && nnMap.size() > 1;
}
/**
* Returns true if HA is using a shared edits directory.
*
* @param conf Configuration
* @return true if HA config is using a shared edits dir, false otherwise.
*/
public static boolean usesSharedEditsDir(Configuration conf) {
return null != conf.get(DFS_NAMENODE_SHARED_EDITS_DIR_KEY);
}
/**
* Get the namenode Id by matching the {@code addressKey}
* with the the address of the local node.
*
* If {@link DFSConfigKeys#DFS_HA_NAMENODE_ID_KEY} is not specifically
* configured, this method determines the namenode Id by matching the local
* node's address with the configured addresses. When a match is found, it
* returns the namenode Id from the corresponding configuration key.
*
* @param conf Configuration
* @return namenode Id on success, null on failure.
* @throws HadoopIllegalArgumentException on error
*/
public static String getNameNodeId(Configuration conf, String nsId) {
String namenodeId = conf.getTrimmed(DFS_HA_NAMENODE_ID_KEY);
if (namenodeId != null) {
return namenodeId;
}
String suffixes[] = DFSUtil.getSuffixIDs(conf, DFS_NAMENODE_RPC_ADDRESS_KEY,
nsId, null, DFSUtil.LOCAL_ADDRESS_MATCHER);
if (suffixes == null) {
String msg = "Configuration " + DFS_NAMENODE_RPC_ADDRESS_KEY +
" must be suffixed with nameservice and namenode ID for HA " +
"configuration.";
throw new HadoopIllegalArgumentException(msg);
}
return suffixes[1];
}
/**
* Similar to
* {@link DFSUtil#getNameServiceIdFromAddress(Configuration,
* InetSocketAddress, String...)}
*/
public static String getNameNodeIdFromAddress(final Configuration conf,
final InetSocketAddress address, String... keys) {
// Configuration with a single namenode and no nameserviceId
String[] ids = DFSUtil.getSuffixIDs(conf, address, keys);
if (ids != null && ids.length > 1) {
return ids[1];
}
return null;
}
/**
* Given the configuration for this node, return a Configuration object for
* the other node in an HA setup.
*
* @param myConf the configuration of this node
* @return the configuration of the other node in an HA setup
*/
public static Configuration getConfForOtherNode(
Configuration myConf) {
String nsId = DFSUtil.getNamenodeNameServiceId(myConf);
Preconditions.checkArgument(nsId != null,
"Could not determine namespace id. Please ensure that this " +
"machine is one of the machines listed as a NN RPC address, " +
"or configure " + DFSConfigKeys.DFS_FEDERATION_NAMESERVICE_ID);
Collection<String> nnIds = DFSUtil.getNameNodeIds(myConf, nsId);
String myNNId = myConf.get(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY);
Preconditions.checkArgument(nnIds != null,
"Could not determine namenode ids in namespace '%s'. " +
"Please configure " +
DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX,
nsId),
nsId);
Preconditions.checkArgument(nnIds.size() == 2,
"Expected exactly 2 NameNodes in namespace '%s'. " +
"Instead, got only %s (NN ids were '%s'",
nsId, nnIds.size(), Joiner.on("','").join(nnIds));
Preconditions.checkState(myNNId != null && !myNNId.isEmpty(),
"Could not determine own NN ID in namespace '%s'. Please " +
"ensure that this node is one of the machines listed as an " +
"NN RPC address, or configure " + DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY,
nsId);
ArrayList<String> nnSet = Lists.newArrayList(nnIds);
nnSet.remove(myNNId);
assert nnSet.size() == 1;
String activeNN = nnSet.get(0);
// Look up the address of the active NN.
Configuration confForOtherNode = new Configuration(myConf);
NameNode.initializeGenericKeys(confForOtherNode, nsId, activeNN);
return confForOtherNode;
}
/**
* This is used only by tests at the moment.
* @return true if the NN should allow read operations while in standby mode.
*/
public static boolean shouldAllowStandbyReads(Configuration conf) {
return conf.getBoolean("dfs.ha.allow.stale.reads", false);
}
public static void setAllowStandbyReads(Configuration conf, boolean val) {
conf.setBoolean("dfs.ha.allow.stale.reads", val);
}
/**
* @return true if the given nameNodeUri appears to be a logical URI.
* This is the case if there is a failover proxy provider configured
* for it in the given configuration.
*/
public static boolean isLogicalUri(
Configuration conf, URI nameNodeUri) {
String host = nameNodeUri.getHost();
String configKey = DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "."
+ host;
return conf.get(configKey) != null;
}
/**
* Parse the HDFS URI out of the provided token.
* @throws IOException if the token is invalid
*/
public static URI getServiceUriFromToken(
Token<DelegationTokenIdentifier> token)
throws IOException {
String tokStr = token.getService().toString();
if (tokStr.startsWith(HA_DT_SERVICE_PREFIX)) {
tokStr = tokStr.replaceFirst(HA_DT_SERVICE_PREFIX, "");
}
try {
return new URI(HdfsConstants.HDFS_URI_SCHEME + "://" +
tokStr);
} catch (URISyntaxException e) {
throw new IOException("Invalid token contents: '" +
tokStr + "'");
}
}
/**
* Get the service name used in the delegation token for the given logical
* HA service.
* @param uri the logical URI of the cluster
* @return the service name
*/
public static Text buildTokenServiceForLogicalUri(URI uri) {
return new Text(HA_DT_SERVICE_PREFIX + uri.getHost());
}
/**
* @return true if this token corresponds to a logical nameservice
* rather than a specific namenode.
*/
public static boolean isTokenForLogicalUri(
Token<DelegationTokenIdentifier> token) {
return token.getService().toString().startsWith(HA_DT_SERVICE_PREFIX);
}
/**
* Locate a delegation token associated with the given HA cluster URI, and if
* one is found, clone it to also represent the underlying namenode address.
* @param ugi the UGI to modify
* @param haUri the logical URI for the cluster
* @param singleNNAddr one of the NNs in the cluster to which the token
* applies
*/
public static void cloneDelegationTokenForLogicalUri(
UserGroupInformation ugi, URI haUri,
InetSocketAddress singleNNAddr) {
Text haService = buildTokenServiceForLogicalUri(haUri);
Token<DelegationTokenIdentifier> haToken =
DelegationTokenSelector.selectHdfsDelegationToken(haService, ugi);
if (haToken == null) {
// no token
return;
}
Token<DelegationTokenIdentifier> specificToken =
new Token<DelegationTokenIdentifier>(haToken);
specificToken.setService(SecurityUtil.buildTokenService(singleNNAddr));
ugi.addToken(specificToken);
LOG.debug("Mapped HA service delegation token for logical URI " +
haUri + " to namenode " + singleNNAddr);
}
}

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
@ -44,6 +45,8 @@ public class HDFSPolicyProvider extends PolicyProvider {
new Service("security.inter.datanode.protocol.acl",
InterDatanodeProtocol.class),
new Service("security.namenode.protocol.acl", NamenodeProtocol.class),
new Service(CommonConfigurationKeys.SECURITY_HA_SERVICE_PROTOCOL_ACL,
HAServiceProtocol.class),
new Service(
CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_REFRESH_POLICY,
RefreshAuthorizationPolicyProtocol.class),

View File

@ -0,0 +1,333 @@
package org.apache.hadoop.hdfs;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.net.InetSocketAddress;
import java.net.URI;
import java.util.HashMap;
import java.util.Map;
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.DFSClient.Conf;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.JournalProtocolTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolPB;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.retry.FailoverProxyProvider;
import org.apache.hadoop.io.retry.RetryPolicies;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.io.retry.RetryProxy;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.RefreshUserMappingsProtocol;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
import org.apache.hadoop.tools.GetUserMappingsProtocol;
import com.google.common.base.Preconditions;
/**
* Create proxy objects to communicate with a remote NN. All remote access to an
* NN should be funneled through this class. Most of the time you'll want to use
* {@link NameNodeProxies#createProxy(Configuration, URI, Class)}, which will
* create either an HA- or non-HA-enabled client proxy as appropriate.
*/
public class NameNodeProxies {
private static final Log LOG = LogFactory.getLog(NameNodeProxies.class);
/**
* Wrapper for a client proxy as well as its associated service ID.
* This is simply used as a tuple-like return type for
* {@link NameNodeProxies#createProxy} and
* {@link NameNodeProxies#createNonHAProxy}.
*/
public static class ProxyAndInfo<PROXYTYPE> {
private final PROXYTYPE proxy;
private final Text dtService;
public ProxyAndInfo(PROXYTYPE proxy, Text dtService) {
this.proxy = proxy;
this.dtService = dtService;
}
public PROXYTYPE getProxy() {
return proxy;
}
public Text getDelegationTokenService() {
return dtService;
}
}
/**
* Creates the namenode proxy with the passed protocol. This will handle
* creation of either HA- or non-HA-enabled proxy objects, depending upon
* if the provided URI is a configured logical URI.
*
* @param conf the configuration containing the required IPC
* properties, client failover configurations, etc.
* @param nameNodeUri the URI pointing either to a specific NameNode
* or to a logical nameservice.
* @param xface the IPC interface which should be created
* @return an object containing both the proxy and the associated
* delegation token service it corresponds to
* @throws IOException if there is an error creating the proxy
**/
@SuppressWarnings("unchecked")
public static <T> ProxyAndInfo<T> createProxy(Configuration conf,
URI nameNodeUri, Class<T> xface) throws IOException {
Class<FailoverProxyProvider<T>> failoverProxyProviderClass =
getFailoverProxyProviderClass(conf, nameNodeUri, xface);
if (failoverProxyProviderClass == null) {
// Non-HA case
return createNonHAProxy(conf, NameNode.getAddress(nameNodeUri), xface,
UserGroupInformation.getCurrentUser(), true);
} else {
// HA case
FailoverProxyProvider<T> failoverProxyProvider = NameNodeProxies
.createFailoverProxyProvider(conf, failoverProxyProviderClass, xface,
nameNodeUri);
Conf config = new Conf(conf);
T proxy = (T) RetryProxy.create(xface, failoverProxyProvider, RetryPolicies
.failoverOnNetworkException(RetryPolicies.TRY_ONCE_THEN_FAIL,
config.maxFailoverAttempts, config.failoverSleepBaseMillis,
config.failoverSleepMaxMillis));
Text dtService = HAUtil.buildTokenServiceForLogicalUri(nameNodeUri);
return new ProxyAndInfo<T>(proxy, dtService);
}
}
/**
* Creates an explicitly non-HA-enabled proxy object. Most of the time you
* don't want to use this, and should instead use {@link NameNodeProxies#createProxy}.
*
* @param conf the configuration object
* @param nnAddr address of the remote NN to connect to
* @param xface the IPC interface which should be created
* @param ugi the user who is making the calls on the proxy object
* @param withRetries certain interfaces have a non-standard retry policy
* @return an object containing both the proxy and the associated
* delegation token service it corresponds to
* @throws IOException
*/
@SuppressWarnings("unchecked")
public static <T> ProxyAndInfo<T> createNonHAProxy(
Configuration conf, InetSocketAddress nnAddr, Class<T> xface,
UserGroupInformation ugi, boolean withRetries) throws IOException {
Text dtService = SecurityUtil.buildTokenService(nnAddr);
T proxy;
if (xface == ClientProtocol.class) {
proxy = (T) createNNProxyWithClientProtocol(nnAddr, conf, ugi,
withRetries);
} else if (xface == JournalProtocol.class) {
proxy = (T) createNNProxyWithJournalProtocol(nnAddr, conf, ugi);
} else if (xface == NamenodeProtocol.class) {
proxy = (T) createNNProxyWithNamenodeProtocol(nnAddr, conf, ugi,
withRetries);
} else if (xface == GetUserMappingsProtocol.class) {
proxy = (T) createNNProxyWithGetUserMappingsProtocol(nnAddr, conf, ugi);
} else if (xface == RefreshUserMappingsProtocol.class) {
proxy = (T) createNNProxyWithRefreshUserMappingsProtocol(nnAddr, conf, ugi);
} else if (xface == RefreshAuthorizationPolicyProtocol.class) {
proxy = (T) createNNProxyWithRefreshAuthorizationPolicyProtocol(nnAddr,
conf, ugi);
} else {
String message = "Upsupported protocol found when creating the proxy " +
"connection to NameNode: " +
((xface != null) ? xface.getClass().getName() : "null");
LOG.error(message);
throw new IllegalStateException(message);
}
return new ProxyAndInfo<T>(proxy, dtService);
}
private static JournalProtocol createNNProxyWithJournalProtocol(
InetSocketAddress address, Configuration conf, UserGroupInformation ugi)
throws IOException {
JournalProtocolPB proxy = (JournalProtocolPB) createNameNodeProxy(address,
conf, ugi, JournalProtocolPB.class);
return new JournalProtocolTranslatorPB(proxy);
}
private static RefreshAuthorizationPolicyProtocol
createNNProxyWithRefreshAuthorizationPolicyProtocol(InetSocketAddress address,
Configuration conf, UserGroupInformation ugi) throws IOException {
RefreshAuthorizationPolicyProtocolPB proxy = (RefreshAuthorizationPolicyProtocolPB)
createNameNodeProxy(address, conf, ugi, RefreshAuthorizationPolicyProtocolPB.class);
return new RefreshAuthorizationPolicyProtocolClientSideTranslatorPB(proxy);
}
private static RefreshUserMappingsProtocol
createNNProxyWithRefreshUserMappingsProtocol(InetSocketAddress address,
Configuration conf, UserGroupInformation ugi) throws IOException {
RefreshUserMappingsProtocolPB proxy = (RefreshUserMappingsProtocolPB)
createNameNodeProxy(address, conf, ugi, RefreshUserMappingsProtocolPB.class);
return new RefreshUserMappingsProtocolClientSideTranslatorPB(proxy);
}
private static GetUserMappingsProtocol createNNProxyWithGetUserMappingsProtocol(
InetSocketAddress address, Configuration conf, UserGroupInformation ugi)
throws IOException {
GetUserMappingsProtocolPB proxy = (GetUserMappingsProtocolPB)
createNameNodeProxy(address, conf, ugi, GetUserMappingsProtocolPB.class);
return new GetUserMappingsProtocolClientSideTranslatorPB(proxy);
}
private static NamenodeProtocol createNNProxyWithNamenodeProtocol(
InetSocketAddress address, Configuration conf, UserGroupInformation ugi,
boolean withRetries) throws IOException {
NamenodeProtocolPB proxy = (NamenodeProtocolPB) createNameNodeProxy(
address, conf, ugi, NamenodeProtocolPB.class);
if (withRetries) { // create the proxy with retries
RetryPolicy timeoutPolicy = RetryPolicies.exponentialBackoffRetry(5, 200,
TimeUnit.MILLISECONDS);
Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap
= new HashMap<Class<? extends Exception>, RetryPolicy>();
RetryPolicy methodPolicy = RetryPolicies.retryByException(timeoutPolicy,
exceptionToPolicyMap);
Map<String, RetryPolicy> methodNameToPolicyMap
= new HashMap<String, RetryPolicy>();
methodNameToPolicyMap.put("getBlocks", methodPolicy);
methodNameToPolicyMap.put("getAccessKeys", methodPolicy);
proxy = (NamenodeProtocolPB) RetryProxy.create(NamenodeProtocolPB.class,
proxy, methodNameToPolicyMap);
}
return new NamenodeProtocolTranslatorPB(proxy);
}
private static ClientProtocol createNNProxyWithClientProtocol(
InetSocketAddress address, Configuration conf, UserGroupInformation ugi,
boolean withRetries) throws IOException {
ClientNamenodeProtocolPB proxy = (ClientNamenodeProtocolPB) NameNodeProxies
.createNameNodeProxy(address, conf, ugi, ClientNamenodeProtocolPB.class);
if (withRetries) { // create the proxy with retries
RetryPolicy createPolicy = RetryPolicies
.retryUpToMaximumCountWithFixedSleep(5,
HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap
= new HashMap<Class<? extends Exception>, RetryPolicy>();
remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
createPolicy);
Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap
= new HashMap<Class<? extends Exception>, RetryPolicy>();
exceptionToPolicyMap.put(RemoteException.class, RetryPolicies
.retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL,
remoteExceptionToPolicyMap));
RetryPolicy methodPolicy = RetryPolicies.retryByException(
RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
Map<String, RetryPolicy> methodNameToPolicyMap
= new HashMap<String, RetryPolicy>();
methodNameToPolicyMap.put("create", methodPolicy);
proxy = (ClientNamenodeProtocolPB) RetryProxy
.create(ClientNamenodeProtocolPB.class, proxy, methodNameToPolicyMap);
}
return new ClientNamenodeProtocolTranslatorPB(proxy);
}
@SuppressWarnings("unchecked")
private static Object createNameNodeProxy(InetSocketAddress address,
Configuration conf, UserGroupInformation ugi, Class xface)
throws IOException {
RPC.setProtocolEngine(conf, xface, ProtobufRpcEngine.class);
Object proxy = RPC.getProxy(xface, RPC.getProtocolVersion(xface), address,
ugi, conf, NetUtils.getDefaultSocketFactory(conf));
return proxy;
}
/** Gets the configured Failover proxy provider's class */
private static <T> Class<FailoverProxyProvider<T>> getFailoverProxyProviderClass(
Configuration conf, URI nameNodeUri, Class<T> xface) throws IOException {
if (nameNodeUri == null) {
return null;
}
String host = nameNodeUri.getHost();
String configKey = DFS_CLIENT_FAILOVER_PROXY_PROVIDER_KEY_PREFIX + "."
+ host;
try {
@SuppressWarnings("unchecked")
Class<FailoverProxyProvider<T>> ret = (Class<FailoverProxyProvider<T>>) conf
.getClass(configKey, null, FailoverProxyProvider.class);
if (ret != null) {
// If we found a proxy provider, then this URI should be a logical NN.
// Given that, it shouldn't have a non-default port number.
int port = nameNodeUri.getPort();
if (port > 0 && port != NameNode.DEFAULT_PORT) {
throw new IOException("Port " + port + " specified in URI "
+ nameNodeUri + " but host '" + host
+ "' is a logical (HA) namenode"
+ " and does not use port information.");
}
}
return ret;
} catch (RuntimeException e) {
if (e.getCause() instanceof ClassNotFoundException) {
throw new IOException("Could not load failover proxy provider class "
+ conf.get(configKey) + " which is configured for authority "
+ nameNodeUri, e);
} else {
throw e;
}
}
}
/** Creates the Failover proxy provider instance*/
@SuppressWarnings("unchecked")
private static <T> FailoverProxyProvider<T> createFailoverProxyProvider(
Configuration conf, Class<FailoverProxyProvider<T>> failoverProxyProviderClass,
Class<T> xface, URI nameNodeUri) throws IOException {
Preconditions.checkArgument(
xface.isAssignableFrom(NamenodeProtocols.class),
"Interface %s is not a NameNode protocol", xface);
try {
Constructor<FailoverProxyProvider<T>> ctor = failoverProxyProviderClass
.getConstructor(Configuration.class, URI.class, Class.class);
FailoverProxyProvider<?> provider = ctor.newInstance(conf, nameNodeUri,
xface);
return (FailoverProxyProvider<T>) provider;
} catch (Exception e) {
String message = "Couldn't create proxy provider " + failoverProxyProviderClass;
if (LOG.isDebugEnabled()) {
LOG.debug(message, e);
}
if (e.getCause() instanceof IOException) {
throw (IOException) e.getCause();
} else {
throw new IOException(message, e);
}
}
}
}

View File

@ -40,7 +40,7 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
* - followed by the invalid replica represented with three -1s;
* - followed by the under-construction replica list where each replica is
* represented by 4 longs: three for the block id, length, generation
* stamp, and the forth for the replica state.
* stamp, and the fourth for the replica state.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
@ -304,4 +304,16 @@ public class BlockListAsLongs implements Iterable<Block> {
blockList[idx+1] = -1;
blockList[idx+2] = -1;
}
public long getMaxGsInBlockList() {
long maxGs = -1;
Iterator<Block> iter = getBlockReportIterator();
while (iter.hasNext()) {
Block b = iter.next();
if (b.getGenerationStamp() > maxGs) {
maxGs = b.getGenerationStamp();
}
}
return maxGs;
}
}

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.retry.Idempotent;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.KerberosInfo;
import org.apache.hadoop.security.token.Token;
@ -114,6 +115,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException If <code>src</code> contains a symlink
* @throws IOException If an I/O error occurred
*/
@Idempotent
public LocatedBlocks getBlockLocations(String src,
long offset,
long length)
@ -125,6 +127,7 @@ public interface ClientProtocol {
* @return a set of server default configuration values
* @throws IOException
*/
@Idempotent
public FsServerDefaults getServerDefaults() throws IOException;
/**
@ -228,6 +231,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException if <code>src</code> contains a symlink
* @throws IOException If an I/O error occurred
*/
@Idempotent
public boolean setReplication(String src, short replication)
throws AccessControlException, DSQuotaExceededException,
FileNotFoundException, SafeModeException, UnresolvedLinkException,
@ -242,6 +246,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException If <code>src</code> contains a symlink
* @throws IOException If an I/O error occurred
*/
@Idempotent
public void setPermission(String src, FsPermission permission)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException;
@ -259,12 +264,13 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException If <code>src</code> contains a symlink
* @throws IOException If an I/O error occurred
*/
@Idempotent
public void setOwner(String src, String username, String groupname)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException;
/**
* The client can give up on a blcok by calling abandonBlock().
* The client can give up on a block by calling abandonBlock().
* The client can then
* either obtain a new block, or complete or abandon the file.
* Any partial writes to the block will be discarded.
@ -331,6 +337,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException If <code>src</code> contains a symlink
* @throws IOException If an I/O error occurred
*/
@Idempotent
public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk,
final DatanodeInfo[] existings, final DatanodeInfo[] excludes,
final int numAdditionalNodes, final String clientName
@ -368,6 +375,7 @@ public interface ClientProtocol {
* locations on datanodes).
* @param blocks Array of located blocks to report
*/
@Idempotent
public void reportBadBlocks(LocatedBlock[] blocks) throws IOException;
///////////////////////////////////////
@ -482,6 +490,7 @@ public interface ClientProtocol {
* RunTimeExceptions:
* @throws InvalidPathException If <code>src</code> is invalid
*/
@Idempotent
public boolean mkdirs(String src, FsPermission masked, boolean createParent)
throws AccessControlException, FileAlreadyExistsException,
FileNotFoundException, NSQuotaExceededException,
@ -502,6 +511,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException If <code>src</code> contains a symlink
* @throws IOException If an I/O error occurred
*/
@Idempotent
public DirectoryListing getListing(String src,
byte[] startAfter,
boolean needLocation)
@ -531,6 +541,7 @@ public interface ClientProtocol {
* @throws AccessControlException permission denied
* @throws IOException If an I/O error occurred
*/
@Idempotent
public void renewLease(String clientName) throws AccessControlException,
IOException;
@ -543,6 +554,7 @@ public interface ClientProtocol {
* @return true if the file is already closed
* @throws IOException
*/
@Idempotent
public boolean recoverLease(String src, String clientName) throws IOException;
public int GET_STATS_CAPACITY_IDX = 0;
@ -554,7 +566,7 @@ public interface ClientProtocol {
/**
* Get a set of statistics about the filesystem.
* Right now, only three values are returned.
* Right now, only seven values are returned.
* <ul>
* <li> [0] contains the total storage capacity of the system, in bytes.</li>
* <li> [1] contains the total used space of the system, in bytes.</li>
@ -567,6 +579,7 @@ public interface ClientProtocol {
* Use public constants like {@link #GET_STATS_CAPACITY_IDX} in place of
* actual numbers to index into the array.
*/
@Idempotent
public long[] getStats() throws IOException;
/**
@ -575,6 +588,7 @@ public interface ClientProtocol {
* Return live datanodes if type is LIVE; dead datanodes if type is DEAD;
* otherwise all datanodes if type is ALL.
*/
@Idempotent
public DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type)
throws IOException;
@ -585,6 +599,7 @@ public interface ClientProtocol {
* @throws IOException
* @throws UnresolvedLinkException if the path contains a symlink.
*/
@Idempotent
public long getPreferredBlockSize(String filename)
throws IOException, UnresolvedLinkException;
@ -700,9 +715,9 @@ public interface ClientProtocol {
* all corrupt files, call this method repeatedly and each time pass in the
* cookie returned from the previous call.
*/
public CorruptFileBlocks
listCorruptFileBlocks(String path, String cookie)
throws IOException;
@Idempotent
public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
throws IOException;
/**
* Dumps namenode data structures into specified file. If the file
@ -719,6 +734,7 @@ public interface ClientProtocol {
* @param bandwidth Blanacer bandwidth in bytes per second for this datanode.
* @throws IOException
*/
@Idempotent
public void setBalancerBandwidth(long bandwidth) throws IOException;
/**
@ -732,6 +748,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException if the path contains a symlink.
* @throws IOException If an I/O error occurred
*/
@Idempotent
public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException;
@ -747,6 +764,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException if <code>src</code> contains a symlink
* @throws IOException If an I/O error occurred
*/
@Idempotent
public HdfsFileStatus getFileLinkInfo(String src)
throws AccessControlException, UnresolvedLinkException, IOException;
@ -759,6 +777,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException if <code>path</code> contains a symlink.
* @throws IOException If an I/O error occurred
*/
@Idempotent
public ContentSummary getContentSummary(String path)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
@ -784,6 +803,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException if the <code>path</code> contains a symlink.
* @throws IOException If an I/O error occurred
*/
@Idempotent
public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
@ -799,6 +819,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException if <code>src</code> contains a symlink.
* @throws IOException If an I/O error occurred
*/
@Idempotent
public void fsync(String src, String client)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
@ -818,6 +839,7 @@ public interface ClientProtocol {
* @throws UnresolvedLinkException if <code>src</code> contains a symlink.
* @throws IOException If an I/O error occurred
*/
@Idempotent
public void setTimes(String src, long mtime, long atime)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
@ -858,6 +880,7 @@ public interface ClientProtocol {
* @throws IOException If the given path does not refer to a symlink
* or an I/O error occurred
*/
@Idempotent
public String getLinkTarget(String path) throws AccessControlException,
FileNotFoundException, IOException;
@ -873,6 +896,7 @@ public interface ClientProtocol {
* @return a located block with a new generation stamp and an access token
* @throws IOException if any error occurs
*/
@Idempotent
public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
String clientName) throws IOException;
@ -896,6 +920,7 @@ public interface ClientProtocol {
* @return Token<DelegationTokenIdentifier>
* @throws IOException
*/
@Idempotent
public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
throws IOException;
@ -906,6 +931,7 @@ public interface ClientProtocol {
* @return the new expiration time
* @throws IOException
*/
@Idempotent
public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
throws IOException;

View File

@ -99,6 +99,14 @@ public class HdfsConstants {
*/
public static final String HDFS_URI_SCHEME = "hdfs";
/**
* A prefix put before the namenode URI inside the "service" field
* of a delgation token, indicating that the URI is a logical (HA)
* URI.
*/
public static final String HA_DT_SERVICE_PREFIX = "ha-hdfs:";
/**
* Please see {@link LayoutVersion} on adding new layout version.
*/

View File

@ -91,7 +91,10 @@ public class LayoutVersion {
STORED_TXIDS(-37, "Transaction IDs are stored in edits log and image files"),
TXID_BASED_LAYOUT(-38, "File names in NN Storage are based on transaction IDs"),
EDITLOG_OP_OPTIMIZATION(-39,
"Use LongWritable and ShortWritable directly instead of ArrayWritable of UTF8");
"Use LongWritable and ShortWritable directly instead of ArrayWritable of UTF8"),
OPTIMIZE_PERSIST_BLOCKS(-40,
"Serialize block lists with delta-encoded variable length ints, " +
"add OP_UPDATE_BLOCKS");
final int lv;
final int ancestorLV;

View File

@ -45,6 +45,7 @@ import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.ProtocolTranslator;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
@ -63,7 +64,8 @@ import com.google.protobuf.ServiceException;
@InterfaceAudience.Private
@InterfaceStability.Stable
public class ClientDatanodeProtocolTranslatorPB implements
ProtocolMetaInterface, ClientDatanodeProtocol, Closeable {
ProtocolMetaInterface, ClientDatanodeProtocol,
ProtocolTranslator, Closeable {
public static final Log LOG = LogFactory
.getLog(ClientDatanodeProtocolTranslatorPB.class);
@ -198,4 +200,9 @@ public class ClientDatanodeProtocolTranslatorPB implements
ClientDatanodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
RPC.getProtocolVersion(ClientDatanodeProtocolPB.class), methodName);
}
@Override
public Object getUnderlyingProxyObject() {
return rpcProxy;
}
}

View File

@ -20,15 +20,10 @@ package org.apache.hadoop.hdfs.protocolPB;
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FileAlreadyExistsException;
@ -49,6 +44,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
import org.apache.hadoop.ipc.ProtocolTranslator;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@ -137,52 +133,14 @@ import com.google.protobuf.ServiceException;
@InterfaceAudience.Private
@InterfaceStability.Stable
public class ClientNamenodeProtocolTranslatorPB implements
ProtocolMetaInterface, ClientProtocol, Closeable {
ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
final private ClientNamenodeProtocolPB rpcProxy;
private static ClientNamenodeProtocolPB createNamenode(
InetSocketAddress nameNodeAddr, Configuration conf,
UserGroupInformation ugi) throws IOException {
RPC.setProtocolEngine(conf, ClientNamenodeProtocolPB.class,
ProtobufRpcEngine.class);
return RPC.getProxy(ClientNamenodeProtocolPB.class,
RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), nameNodeAddr, ugi, conf,
NetUtils.getSocketFactory(conf, ClientNamenodeProtocolPB.class));
public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy)
throws IOException {
rpcProxy = proxy;
}
/** Create a {@link NameNode} proxy */
static ClientNamenodeProtocolPB createNamenodeWithRetry(
ClientNamenodeProtocolPB rpcNamenode) {
RetryPolicy createPolicy = RetryPolicies
.retryUpToMaximumCountWithFixedSleep(5,
HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap
= new HashMap<Class<? extends Exception>, RetryPolicy>();
remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
createPolicy);
Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
new HashMap<Class<? extends Exception>, RetryPolicy>();
exceptionToPolicyMap.put(RemoteException.class, RetryPolicies
.retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL,
remoteExceptionToPolicyMap));
RetryPolicy methodPolicy = RetryPolicies.retryByException(
RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
Map<String, RetryPolicy> methodNameToPolicyMap = new HashMap<String, RetryPolicy>();
methodNameToPolicyMap.put("create", methodPolicy);
return (ClientNamenodeProtocolPB) RetryProxy.create(
ClientNamenodeProtocolPB.class, rpcNamenode, methodNameToPolicyMap);
}
public ClientNamenodeProtocolTranslatorPB(InetSocketAddress nameNodeAddr,
Configuration conf, UserGroupInformation ugi) throws IOException {
rpcProxy = createNamenodeWithRetry(createNamenode(nameNodeAddr, conf, ugi));
}
public void close() {
RPC.stopProxy(rpcProxy);
}
@ -866,4 +824,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
ClientNamenodeProtocolPB.class, RpcKind.RPC_PROTOCOL_BUFFER,
RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
}
@Override
public Object getUnderlyingProxyObject() {
return rpcProxy;
}
}

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeComm
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.HeartbeatResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.NNHAStatusHeartbeatProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ProcessUpgradeResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterDatanodeRequestProto;
@ -55,6 +56,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
@ -161,7 +163,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
}
@Override
public DatanodeCommand[] sendHeartbeat(DatanodeRegistration registration,
public HeartbeatResponse sendHeartbeat(DatanodeRegistration registration,
StorageReport[] reports, int xmitsInProgress, int xceiverCount,
int failedVolumes) throws IOException {
HeartbeatRequestProto.Builder builder = HeartbeatRequestProto.newBuilder()
@ -184,7 +186,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
cmds[index] = PBHelper.convert(p);
index++;
}
return cmds;
return new HeartbeatResponse(cmds, PBHelper.convert(resp.getHaStatus()));
}
@Override

View File

@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
@ -103,7 +104,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
@Override
public HeartbeatResponseProto sendHeartbeat(RpcController controller,
HeartbeatRequestProto request) throws ServiceException {
DatanodeCommand[] cmds = null;
HeartbeatResponse response;
try {
List<StorageReportProto> list = request.getReportsList();
StorageReport[] report = new StorageReport[list.size()];
@ -113,7 +114,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
p.getBlockPoolUsed());
}
cmds = impl.sendHeartbeat(PBHelper.convert(request.getRegistration()),
response = impl.sendHeartbeat(PBHelper.convert(request.getRegistration()),
report, request.getXmitsInProgress(), request.getXceiverCount(),
request.getFailedVolumes());
} catch (IOException e) {
@ -121,6 +122,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
}
HeartbeatResponseProto.Builder builder = HeartbeatResponseProto
.newBuilder();
DatanodeCommand[] cmds = response.getCommands();
if (cmds != null) {
for (int i = 0; i < cmds.length; i++) {
if (cmds[i] != null) {
@ -128,6 +130,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
}
}
}
builder.setHaStatus(PBHelper.convert(response.getNameNodeHaState()));
return builder.build();
}

View File

@ -20,22 +20,15 @@ package org.apache.hadoop.hdfs.protocolPB;
import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetGroupsForUserRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetGroupsForUserResponseProto;
import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.tools.GetUserMappingsProtocol;
import com.google.protobuf.RpcController;
@ -47,16 +40,10 @@ public class GetUserMappingsProtocolClientSideTranslatorPB implements
/** RpcController is not used and hence is set to null */
private final static RpcController NULL_CONTROLLER = null;
private final GetUserMappingsProtocolPB rpcProxy;
public GetUserMappingsProtocolClientSideTranslatorPB(
InetSocketAddress nameNodeAddr, UserGroupInformation ugi,
Configuration conf) throws IOException {
RPC.setProtocolEngine(conf, GetUserMappingsProtocolPB.class,
ProtobufRpcEngine.class);
rpcProxy = RPC.getProxy(GetUserMappingsProtocolPB.class,
RPC.getProtocolVersion(GetUserMappingsProtocolPB.class),
NameNode.getAddress(conf), ugi, conf,
NetUtils.getSocketFactory(conf, GetUserMappingsProtocol.class));
GetUserMappingsProtocolPB rpcProxy) {
this.rpcProxy = rpcProxy;
}
@Override

View File

@ -19,17 +19,14 @@ package org.apache.hadoop.hdfs.protocolPB;
import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.StartLogSegmentRequestProto;
import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.ProtocolSignature;
@ -52,12 +49,9 @@ public class JournalProtocolTranslatorPB implements ProtocolMetaInterface,
/** RpcController is not used and hence is set to null */
private final static RpcController NULL_CONTROLLER = null;
private final JournalProtocolPB rpcProxy;
public JournalProtocolTranslatorPB(InetSocketAddress nameNodeAddr,
Configuration conf) throws IOException {
RPC.setProtocolEngine(conf, JournalProtocolPB.class, ProtobufRpcEngine.class);
rpcProxy = RPC.getProxy(JournalProtocolPB.class,
RPC.getProtocolVersion(JournalProtocolPB.class), nameNodeAddr, conf);
public JournalProtocolTranslatorPB(JournalProtocolPB rpcProxy) {
this.rpcProxy = rpcProxy;
}
@Override

View File

@ -19,11 +19,9 @@ package org.apache.hadoop.hdfs.protocolPB;
import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
@ -47,14 +45,11 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
@ -84,15 +79,6 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
VersionRequestProto.newBuilder().build();
final private NamenodeProtocolPB rpcProxy;
public NamenodeProtocolTranslatorPB(InetSocketAddress nameNodeAddr,
Configuration conf, UserGroupInformation ugi) throws IOException {
RPC.setProtocolEngine(conf, NamenodeProtocolPB.class,
ProtobufRpcEngine.class);
rpcProxy = RPC.getProxy(NamenodeProtocolPB.class,
RPC.getProtocolVersion(NamenodeProtocolPB.class), nameNodeAddr, ugi,
conf, NetUtils.getSocketFactory(conf, NamenodeProtocolPB.class));
}
public NamenodeProtocolTranslatorPB(NamenodeProtocolPB rpcProxy) {
this.rpcProxy = rpcProxy;
@ -137,7 +123,6 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol,
}
@Override
@SuppressWarnings("deprecation")
public CheckpointSignature rollEditLog() throws IOException {
try {
return PBHelper.convert(rpcProxy.rollEditLog(NULL_CONTROLLER,

View File

@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeStor
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeStorageProto.StorageState;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.NNHAStatusHeartbeatProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReportProto;
@ -119,7 +120,9 @@ import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
@ -819,6 +822,23 @@ public class PBHelper {
ReceivedDeletedBlockInfoProto.Builder builder =
ReceivedDeletedBlockInfoProto.newBuilder();
ReceivedDeletedBlockInfoProto.BlockStatus status;
switch (receivedDeletedBlockInfo.getStatus()) {
case RECEIVING_BLOCK:
status = ReceivedDeletedBlockInfoProto.BlockStatus.RECEIVING;
break;
case RECEIVED_BLOCK:
status = ReceivedDeletedBlockInfoProto.BlockStatus.RECEIVED;
break;
case DELETED_BLOCK:
status = ReceivedDeletedBlockInfoProto.BlockStatus.DELETED;
break;
default:
throw new IllegalArgumentException("Bad status: " +
receivedDeletedBlockInfo.getStatus());
}
builder.setStatus(status);
if (receivedDeletedBlockInfo.getDelHints() != null) {
builder.setDeleteHint(receivedDeletedBlockInfo.getDelHints());
}
@ -850,7 +870,21 @@ public class PBHelper {
public static ReceivedDeletedBlockInfo convert(
ReceivedDeletedBlockInfoProto proto) {
return new ReceivedDeletedBlockInfo(PBHelper.convert(proto.getBlock()),
ReceivedDeletedBlockInfo.BlockStatus status = null;
switch (proto.getStatus()) {
case RECEIVING:
status = BlockStatus.RECEIVING_BLOCK;
break;
case RECEIVED:
status = BlockStatus.RECEIVED_BLOCK;
break;
case DELETED:
status = BlockStatus.DELETED_BLOCK;
break;
}
return new ReceivedDeletedBlockInfo(
PBHelper.convert(proto.getBlock()),
status,
proto.hasDeleteHint() ? proto.getDeleteHint() : null);
}
@ -1245,6 +1279,37 @@ public class PBHelper {
build();
}
public static NNHAStatusHeartbeat convert(NNHAStatusHeartbeatProto s) {
if (s == null) return null;
switch (s.getState()) {
case ACTIVE:
return new NNHAStatusHeartbeat(NNHAStatusHeartbeat.State.ACTIVE, s.getTxid());
case STANDBY:
return new NNHAStatusHeartbeat(NNHAStatusHeartbeat.State.STANDBY, s.getTxid());
default:
throw new IllegalArgumentException("Unexpected NNHAStatusHeartbeat.State:" + s.getState());
}
}
public static NNHAStatusHeartbeatProto convert(NNHAStatusHeartbeat hb) {
if (hb == null) return null;
NNHAStatusHeartbeatProto.Builder builder =
NNHAStatusHeartbeatProto.newBuilder();
switch (hb.getState()) {
case ACTIVE:
builder.setState(NNHAStatusHeartbeatProto.State.ACTIVE);
break;
case STANDBY:
builder.setState(NNHAStatusHeartbeatProto.State.STANDBY);
break;
default:
throw new IllegalArgumentException("Unexpected NNHAStatusHeartbeat.State:" +
hb.getState());
}
builder.setTxid(hb.getTxId());
return builder.build();
}
public static DatanodeStorageProto convert(DatanodeStorage s) {
return DatanodeStorageProto.newBuilder()
.setState(PBHelper.convert(s.getState()))

View File

@ -20,21 +20,15 @@ package org.apache.hadoop.hdfs.protocolPB;
import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.proto.RefreshAuthorizationPolicyProtocolProtos.RefreshServiceAclRequestProto;
import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
import com.google.protobuf.RpcController;
@ -46,16 +40,10 @@ public class RefreshAuthorizationPolicyProtocolClientSideTranslatorPB implements
/** RpcController is not used and hence is set to null */
private final static RpcController NULL_CONTROLLER = null;
private final RefreshAuthorizationPolicyProtocolPB rpcProxy;
public RefreshAuthorizationPolicyProtocolClientSideTranslatorPB(
InetSocketAddress nameNodeAddr, UserGroupInformation ugi,
Configuration conf) throws IOException {
RPC.setProtocolEngine(conf, RefreshAuthorizationPolicyProtocolPB.class,
ProtobufRpcEngine.class);
rpcProxy = RPC.getProxy(RefreshAuthorizationPolicyProtocolPB.class,
RPC.getProtocolVersion(RefreshAuthorizationPolicyProtocolPB.class),
NameNode.getAddress(conf), ugi, conf,
NetUtils.getSocketFactory(conf, RefreshAuthorizationPolicyProtocol.class));
RefreshAuthorizationPolicyProtocolPB rpcProxy) {
this.rpcProxy = rpcProxy;
}
@Override

View File

@ -20,23 +20,17 @@ package org.apache.hadoop.hdfs.protocolPB;
import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshSuperUserGroupsConfigurationRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshUserToGroupsMappingsRequestProto;
import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.RefreshUserMappingsProtocol;
import org.apache.hadoop.security.UserGroupInformation;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
@ -47,16 +41,10 @@ public class RefreshUserMappingsProtocolClientSideTranslatorPB implements
/** RpcController is not used and hence is set to null */
private final static RpcController NULL_CONTROLLER = null;
private final RefreshUserMappingsProtocolPB rpcProxy;
public RefreshUserMappingsProtocolClientSideTranslatorPB(
InetSocketAddress nameNodeAddr, UserGroupInformation ugi,
Configuration conf) throws IOException {
RPC.setProtocolEngine(conf, RefreshUserMappingsProtocolPB.class,
ProtobufRpcEngine.class);
rpcProxy = RPC.getProxy(RefreshUserMappingsProtocolPB.class,
RPC.getProtocolVersion(RefreshUserMappingsProtocolPB.class),
NameNode.getAddress(conf), ugi, conf,
NetUtils.getSocketFactory(conf, RefreshUserMappingsProtocol.class));
RefreshUserMappingsProtocolPB rpcProxy) {
this.rpcProxy = rpcProxy;
}
@Override

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.security.token.delegation;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.net.InetSocketAddress;
import java.util.Iterator;
@ -283,7 +284,18 @@ public class DelegationTokenSecretManager
@Override //AbstractDelegationTokenManager
protected void logUpdateMasterKey(DelegationKey key)
throws IOException {
namesystem.logUpdateMasterKey(key);
synchronized (noInterruptsLock) {
// The edit logging code will fail catastrophically if it
// is interrupted during a logSync, since the interrupt
// closes the edit log files. Doing this inside the
// above lock and then checking interruption status
// prevents this bug.
if (Thread.interrupted()) {
throw new InterruptedIOException(
"Interrupted before updating master key");
}
namesystem.logUpdateMasterKey(key);
}
}
/** A utility method for creating credentials. */

View File

@ -59,6 +59,11 @@ public class DelegationTokenSelector
new InetSocketAddress(nnAddr.getHostName(), nnRpcPort));
return INSTANCE.selectToken(serviceName, ugi.getTokens());
}
public static Token<DelegationTokenIdentifier> selectHdfsDelegationToken(
Text serviceName, UserGroupInformation ugi) {
return INSTANCE.selectToken(serviceName, ugi.getTokens());
}
public DelegationTokenSelector() {
super(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);

View File

@ -24,8 +24,8 @@ import java.io.BufferedOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.URI;
import java.text.DateFormat;
import java.util.ArrayList;
import java.util.Arrays;
@ -1379,7 +1379,7 @@ public class Balancer {
* for each namenode,
* execute a {@link Balancer} to work through all datanodes once.
*/
static int run(List<InetSocketAddress> namenodes, final Parameters p,
static int run(Collection<URI> namenodes, final Parameters p,
Configuration conf) throws IOException, InterruptedException {
final long sleeptime = 2000*conf.getLong(
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
@ -1393,8 +1393,8 @@ public class Balancer {
final List<NameNodeConnector> connectors
= new ArrayList<NameNodeConnector>(namenodes.size());
try {
for(InetSocketAddress isa : namenodes) {
connectors.add(new NameNodeConnector(isa, conf));
for (URI uri : namenodes) {
connectors.add(new NameNodeConnector(uri, conf));
}
boolean done = false;
@ -1476,7 +1476,7 @@ public class Balancer {
try {
checkReplicationPolicyCompatibility(conf);
final List<InetSocketAddress> namenodes = DFSUtil.getNNServiceRpcAddresses(conf);
final Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
return Balancer.run(namenodes, parse(args), conf);
} catch (IOException e) {
System.out.println(e + ". Exiting ...");

View File

@ -21,38 +21,25 @@ import java.io.DataOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.URI;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.retry.RetryPolicies;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.io.retry.RetryProxy;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Daemon;
@ -64,7 +51,7 @@ class NameNodeConnector {
private static final Log LOG = Balancer.LOG;
private static final Path BALANCER_ID_PATH = new Path("/system/balancer.id");
final InetSocketAddress namenodeAddress;
final URI nameNodeUri;
final String blockpoolID;
final NamenodeProtocol namenode;
@ -78,12 +65,17 @@ class NameNodeConnector {
private BlockTokenSecretManager blockTokenSecretManager;
private Daemon keyupdaterthread; // AccessKeyUpdater thread
NameNodeConnector(InetSocketAddress namenodeAddress, Configuration conf
) throws IOException {
this.namenodeAddress = namenodeAddress;
this.namenode = createNamenode(namenodeAddress, conf);
this.client = DFSUtil.createNamenode(conf);
this.fs = FileSystem.get(NameNode.getUri(namenodeAddress), conf);
NameNodeConnector(URI nameNodeUri,
Configuration conf) throws IOException {
this.nameNodeUri = nameNodeUri;
this.namenode =
NameNodeProxies.createProxy(conf, nameNodeUri, NamenodeProtocol.class)
.getProxy();
this.client =
NameNodeProxies.createProxy(conf, nameNodeUri, ClientProtocol.class)
.getProxy();
this.fs = FileSystem.get(nameNodeUri, conf);
final NamespaceInfo namespaceinfo = namenode.versionRequest();
this.blockpoolID = namespaceinfo.getBlockPoolID();
@ -188,38 +180,11 @@ class NameNodeConnector {
@Override
public String toString() {
return getClass().getSimpleName() + "[namenodeAddress=" + namenodeAddress
return getClass().getSimpleName() + "[namenodeUri=" + nameNodeUri
+ ", id=" + blockpoolID
+ "]";
}
/** Build a NamenodeProtocol connection to the namenode and
* set up the retry policy
*/
private static NamenodeProtocol createNamenode(InetSocketAddress address,
Configuration conf) throws IOException {
RetryPolicy timeoutPolicy = RetryPolicies.exponentialBackoffRetry(
5, 200, TimeUnit.MILLISECONDS);
Map<Class<? extends Exception>,RetryPolicy> exceptionToPolicyMap =
new HashMap<Class<? extends Exception>, RetryPolicy>();
RetryPolicy methodPolicy = RetryPolicies.retryByException(
timeoutPolicy, exceptionToPolicyMap);
Map<String,RetryPolicy> methodNameToPolicyMap =
new HashMap<String, RetryPolicy>();
methodNameToPolicyMap.put("getBlocks", methodPolicy);
methodNameToPolicyMap.put("getAccessKeys", methodPolicy);
RPC.setProtocolEngine(conf, NamenodeProtocolPB.class,
ProtobufRpcEngine.class);
NamenodeProtocolPB proxy = RPC.getProxy(NamenodeProtocolPB.class,
RPC.getProtocolVersion(NamenodeProtocolPB.class), address,
UserGroupInformation.getCurrentUser(), conf,
NetUtils.getDefaultSocketFactory(conf));
NamenodeProtocolPB retryProxy = (NamenodeProtocolPB) RetryProxy.create(
NamenodeProtocolPB.class, proxy, methodNameToPolicyMap);
return new NamenodeProtocolTranslatorPB(retryProxy);
}
/**
* Periodically updates access keys.
*/

View File

@ -183,7 +183,7 @@ public class BlockInfo extends Block implements
/**
* Count the number of data-nodes the block belongs to.
*/
int numNodes() {
public int numNodes() {
assert this.triplets != null : "BlockInfo is not initialized";
assert triplets.length % 3 == 0 : "Malformed BlockInfo";
for(int idx = getCapacity()-1; idx >= 0; idx--) {

View File

@ -63,7 +63,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
initialize(conf, stats, clusterMap);
}
BlockPlacementPolicyDefault() {
protected BlockPlacementPolicyDefault() {
}
@Override

View File

@ -94,6 +94,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
boolean contains(E e) {
return blockq.contains(e);
}
synchronized void clear() {
blockq.clear();
}
}
private volatile BlockInfo blockList = null;
@ -103,6 +107,24 @@ public class DatanodeDescriptor extends DatanodeInfo {
public boolean isAlive = false;
public boolean needKeyUpdate = false;
/**
* Set to false on any NN failover, and reset to true
* whenever a block report is received.
*/
private boolean heartbeatedSinceFailover = false;
/**
* At startup or at any failover, the DNs in the cluster may
* have pending block deletions from a previous incarnation
* of the NameNode. Thus, we consider their block contents
* stale until we have received a block report. When a DN
* is considered stale, any replicas on it are transitively
* considered stale. If any block has at least one stale replica,
* then no invalidations will be processed for this block.
* See HDFS-1972.
*/
private boolean blockContentsStale = true;
// A system administrator can tune the balancer bandwidth parameter
// (dfs.balance.bandwidthPerSec) dynamically by calling
// "dfsadmin -setBalanacerBandwidth <newbandwidth>", at which point the
@ -129,6 +151,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
private long lastBlocksScheduledRollTime = 0;
private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min
private int volumeFailures = 0;
/** Set to false after processing first block report */
private boolean firstBlockReport = true;
/**
* When set to true, the node is not in include list and is not allowed
* to communicate with the namenode
@ -281,6 +307,14 @@ public class DatanodeDescriptor extends DatanodeInfo {
this.invalidateBlocks.clear();
this.volumeFailures = 0;
}
public void clearBlockQueues() {
synchronized (invalidateBlocks) {
this.invalidateBlocks.clear();
this.recoverBlocks.clear();
this.replicateBlocks.clear();
}
}
public int numBlocks() {
return numBlocks;
@ -298,6 +332,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
this.lastUpdate = System.currentTimeMillis();
this.xceiverCount = xceiverCount;
this.volumeFailures = volFailures;
this.heartbeatedSinceFailover = true;
rollBlocksScheduled(lastUpdate);
}
@ -564,5 +599,41 @@ public class DatanodeDescriptor extends DatanodeInfo {
this.bandwidth = bandwidth;
}
public boolean areBlockContentsStale() {
return blockContentsStale;
}
public void markStaleAfterFailover() {
heartbeatedSinceFailover = false;
blockContentsStale = true;
}
public void receivedBlockReport() {
if (heartbeatedSinceFailover) {
blockContentsStale = false;
}
firstBlockReport = false;
}
boolean isFirstBlockReport() {
return firstBlockReport;
}
@Override
public String dumpDatanode() {
StringBuilder sb = new StringBuilder(super.dumpDatanode());
int repl = replicateBlocks.size();
if (repl > 0) {
sb.append(" ").append(repl).append(" blocks to be replicated;");
}
int inval = invalidateBlocks.size();
if (inval > 0) {
sb.append(" ").append(inval).append(" blocks to be invalidated;");
}
int recover = recoverBlocks.size();
if (recover > 0) {
sb.append(" ").append(recover).append(" blocks to be recovered;");
}
return sb.toString();
}
}

View File

@ -923,7 +923,7 @@ public class DatanodeManager {
}
}
return null;
return new DatanodeCommand[0];
}
/**
@ -947,4 +947,27 @@ public class DatanodeManager {
}
}
}
public void markAllDatanodesStale() {
LOG.info("Marking all datandoes as stale");
synchronized (datanodeMap) {
for (DatanodeDescriptor dn : datanodeMap.values()) {
dn.markStaleAfterFailover();
}
}
}
/**
* Clear any actions that are queued up to be sent to the DNs
* on their next heartbeats. This includes block invalidations,
* recoveries, and replication requests.
*/
public void clearPendingQueues() {
synchronized (datanodeMap) {
for (DatanodeDescriptor dn : datanodeMap.values()) {
dn.clearBlockQueues();
}
}
}
}

View File

@ -160,4 +160,9 @@ class InvalidateBlocks {
numBlocks -= toInvalidate.size();
return toInvalidate;
}
synchronized void clear() {
node2blocks.clear();
numBlocks = 0;
}
}

View File

@ -26,20 +26,22 @@ public class NumberReplicas {
private int decommissionedReplicas;
private int corruptReplicas;
private int excessReplicas;
private int replicasOnStaleNodes;
NumberReplicas() {
initialize(0, 0, 0, 0);
initialize(0, 0, 0, 0, 0);
}
NumberReplicas(int live, int decommissioned, int corrupt, int excess) {
initialize(live, decommissioned, corrupt, excess);
NumberReplicas(int live, int decommissioned, int corrupt, int excess, int stale) {
initialize(live, decommissioned, corrupt, excess, stale);
}
void initialize(int live, int decommissioned, int corrupt, int excess) {
void initialize(int live, int decommissioned, int corrupt, int excess, int stale) {
liveReplicas = live;
decommissionedReplicas = decommissioned;
corruptReplicas = corrupt;
excessReplicas = excess;
replicasOnStaleNodes = stale;
}
public int liveReplicas() {
@ -54,4 +56,13 @@ public class NumberReplicas {
public int excessReplicas() {
return excessReplicas;
}
/**
* @return the number of replicas which are on stale nodes.
* This is not mutually exclusive with the other counts -- ie a
* replica may count as both "live" and "stale".
*/
public int replicasOnStaleNodes() {
return replicasOnStaleNodes;
}
}

View File

@ -0,0 +1,134 @@
/**
* 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.server.blockmanagement;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
/**
* In the Standby Node, we can receive messages about blocks
* before they are actually available in the namespace, or while
* they have an outdated state in the namespace. In those cases,
* we queue those block-related messages in this structure.
* */
class PendingDataNodeMessages {
Map<Block, Queue<ReportedBlockInfo>> queueByBlockId =
Maps.newHashMap();
private int count = 0;
static class ReportedBlockInfo {
private final Block block;
private final DatanodeDescriptor dn;
private final ReplicaState reportedState;
ReportedBlockInfo(DatanodeDescriptor dn, Block block,
ReplicaState reportedState) {
this.dn = dn;
this.block = block;
this.reportedState = reportedState;
}
Block getBlock() {
return block;
}
DatanodeDescriptor getNode() {
return dn;
}
ReplicaState getReportedState() {
return reportedState;
}
@Override
public String toString() {
return "ReportedBlockInfo [block=" + block + ", dn=" + dn
+ ", reportedState=" + reportedState + "]";
}
}
void enqueueReportedBlock(DatanodeDescriptor dn, Block block,
ReplicaState reportedState) {
block = new Block(block);
getBlockQueue(block).add(
new ReportedBlockInfo(dn, block, reportedState));
count++;
}
/**
* @return any messages that were previously queued for the given block,
* or null if no messages were queued.
*/
Queue<ReportedBlockInfo> takeBlockQueue(Block block) {
Queue<ReportedBlockInfo> queue = queueByBlockId.remove(block);
if (queue != null) {
count -= queue.size();
}
return queue;
}
private Queue<ReportedBlockInfo> getBlockQueue(Block block) {
Queue<ReportedBlockInfo> queue = queueByBlockId.get(block);
if (queue == null) {
queue = Lists.newLinkedList();
queueByBlockId.put(block, queue);
}
return queue;
}
public int count() {
return count ;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
for (Map.Entry<Block, Queue<ReportedBlockInfo>> entry :
queueByBlockId.entrySet()) {
sb.append("Block " + entry.getKey() + ":\n");
for (ReportedBlockInfo rbi : entry.getValue()) {
sb.append(" ").append(rbi).append("\n");
}
}
return sb.toString();
}
public Iterable<ReportedBlockInfo> takeAll() {
List<ReportedBlockInfo> rbis = Lists.newArrayListWithCapacity(
count);
for (Queue<ReportedBlockInfo> q : queueByBlockId.values()) {
rbis.addAll(q);
}
queueByBlockId.clear();
count = 0;
return rbis;
}
}

View File

@ -104,6 +104,14 @@ class PendingReplicationBlocks {
}
}
public void clear() {
synchronized (pendingReplications) {
pendingReplications.clear();
timedOutItems.clear();
}
}
/**
* The total number of blocks that are undergoing replication
*/

View File

@ -568,7 +568,7 @@ public abstract class Storage extends StorageInfo {
* <p> Locking is not supported by all file systems.
* E.g., NFS does not consistently support exclusive locks.
*
* <p> If locking is supported we guarantee exculsive access to the
* <p> If locking is supported we guarantee exclusive access to the
* storage directory. Otherwise, no guarantee is given.
*
* @throws IOException if locking fails

View File

@ -23,6 +23,7 @@ import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -97,9 +98,9 @@ public final class Util {
* @param names collection of strings to convert to URIs
* @return collection of URIs
*/
public static Collection<URI> stringCollectionAsURIs(
public static List<URI> stringCollectionAsURIs(
Collection<String> names) {
Collection<URI> uris = new ArrayList<URI>(names.size());
List<URI> uris = new ArrayList<URI>(names.size());
for(String name : names) {
try {
uris.add(stringAsURI(name));

View File

@ -0,0 +1,730 @@
/**
* 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.server.datanode;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.SocketTimeoutException;
import java.net.URI;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.StringUtils;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Maps;
/**
* A thread per active or standby namenode to perform:
* <ul>
* <li> Pre-registration handshake with namenode</li>
* <li> Registration with namenode</li>
* <li> Send periodic heartbeats to the namenode</li>
* <li> Handle commands received from the namenode</li>
* </ul>
*/
@InterfaceAudience.Private
class BPServiceActor implements Runnable {
static final Log LOG = DataNode.LOG;
final InetSocketAddress nnAddr;
BPOfferService bpos;
long lastBlockReport = 0;
long lastDeletedReport = 0;
boolean resetBlockReportTime = true;
Thread bpThread;
DatanodeProtocolClientSideTranslatorPB bpNamenode;
private long lastHeartbeat = 0;
private volatile boolean initialized = false;
/**
* Between block reports (which happen on the order of once an hour) the
* DN reports smaller incremental changes to its block list. This map,
* keyed by block ID, contains the pending changes which have yet to be
* reported to the NN. Access should be synchronized on this object.
*/
private final Map<Long, ReceivedDeletedBlockInfo> pendingIncrementalBR
= Maps.newHashMap();
private volatile int pendingReceivedRequests = 0;
private volatile boolean shouldServiceRun = true;
private final DataNode dn;
private final DNConf dnConf;
private DatanodeRegistration bpRegistration;
BPServiceActor(InetSocketAddress nnAddr, BPOfferService bpos) {
this.bpos = bpos;
this.dn = bpos.getDataNode();
this.nnAddr = nnAddr;
this.dnConf = dn.getDnConf();
}
/**
* returns true if BP thread has completed initialization of storage
* and has registered with the corresponding namenode
* @return true if initialized
*/
boolean isInitialized() {
return initialized;
}
boolean isAlive() {
return shouldServiceRun && bpThread.isAlive();
}
@Override
public String toString() {
return bpos.toString() + " service to " + nnAddr;
}
InetSocketAddress getNNSocketAddress() {
return nnAddr;
}
/**
* Used to inject a spy NN in the unit tests.
*/
@VisibleForTesting
void setNameNode(DatanodeProtocolClientSideTranslatorPB dnProtocol) {
bpNamenode = dnProtocol;
}
@VisibleForTesting
DatanodeProtocolClientSideTranslatorPB getNameNodeProxy() {
return bpNamenode;
}
/**
* Perform the first part of the handshake with the NameNode.
* This calls <code>versionRequest</code> to determine the NN's
* namespace and version info. It automatically retries until
* the NN responds or the DN is shutting down.
*
* @return the NamespaceInfo
*/
@VisibleForTesting
NamespaceInfo retrieveNamespaceInfo() throws IOException {
NamespaceInfo nsInfo = null;
while (shouldRun()) {
try {
nsInfo = bpNamenode.versionRequest();
LOG.debug(this + " received versionRequest response: " + nsInfo);
break;
} catch(SocketTimeoutException e) { // namenode is busy
LOG.warn("Problem connecting to server: " + nnAddr);
} catch(IOException e ) { // namenode is not available
LOG.warn("Problem connecting to server: " + nnAddr);
}
// try again in a second
sleepAndLogInterrupts(5000, "requesting version info from NN");
}
if (nsInfo != null) {
checkNNVersion(nsInfo);
} else {
throw new IOException("DN shut down before block pool connected");
}
return nsInfo;
}
private void checkNNVersion(NamespaceInfo nsInfo)
throws IncorrectVersionException {
// build and layout versions should match
String nsBuildVer = nsInfo.getBuildVersion();
String stBuildVer = Storage.getBuildVersion();
if (!nsBuildVer.equals(stBuildVer)) {
LOG.warn("Data-node and name-node Build versions must be the same. " +
"Namenode build version: " + nsBuildVer + "Datanode " +
"build version: " + stBuildVer);
throw new IncorrectVersionException(nsBuildVer, "namenode", stBuildVer);
}
if (HdfsConstants.LAYOUT_VERSION != nsInfo.getLayoutVersion()) {
LOG.warn("Data-node and name-node layout versions must be the same." +
" Expected: "+ HdfsConstants.LAYOUT_VERSION +
" actual "+ nsInfo.getLayoutVersion());
throw new IncorrectVersionException(
nsInfo.getLayoutVersion(), "namenode");
}
}
private void connectToNNAndHandshake() throws IOException {
// get NN proxy
bpNamenode = dn.connectToNN(nnAddr);
// First phase of the handshake with NN - get the namespace
// info.
NamespaceInfo nsInfo = retrieveNamespaceInfo();
// Verify that this matches the other NN in this HA pair.
// This also initializes our block pool in the DN if we are
// the first NN connection for this BP.
bpos.verifyAndSetNamespaceInfo(nsInfo);
// Second phase of the handshake with the NN.
register();
}
/**
* This methods arranges for the data node to send the block report at
* the next heartbeat.
*/
void scheduleBlockReport(long delay) {
if (delay > 0) { // send BR after random delay
lastBlockReport = System.currentTimeMillis()
- ( dnConf.blockReportInterval - DFSUtil.getRandom().nextInt((int)(delay)));
} else { // send at next heartbeat
lastBlockReport = lastHeartbeat - dnConf.blockReportInterval;
}
resetBlockReportTime = true; // reset future BRs for randomness
}
void reportBadBlocks(ExtendedBlock block) {
DatanodeInfo[] dnArr = { new DatanodeInfo(bpRegistration) };
LocatedBlock[] blocks = { new LocatedBlock(block, dnArr) };
try {
bpNamenode.reportBadBlocks(blocks);
} catch (IOException e){
/* One common reason is that NameNode could be in safe mode.
* Should we keep on retrying in that case?
*/
LOG.warn("Failed to report bad block " + block + " to namenode : "
+ " Exception", e);
}
}
/**
* Report received blocks and delete hints to the Namenode
*
* @throws IOException
*/
private void reportReceivedDeletedBlocks() throws IOException {
// check if there are newly received blocks
ReceivedDeletedBlockInfo[] receivedAndDeletedBlockArray = null;
synchronized (pendingIncrementalBR) {
int numBlocks = pendingIncrementalBR.size();
if (numBlocks > 0) {
//
// Send newly-received and deleted blockids to namenode
//
receivedAndDeletedBlockArray = pendingIncrementalBR
.values().toArray(new ReceivedDeletedBlockInfo[numBlocks]);
}
pendingIncrementalBR.clear();
}
if (receivedAndDeletedBlockArray != null) {
StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
bpRegistration.getStorageID(), receivedAndDeletedBlockArray) };
boolean success = false;
try {
bpNamenode.blockReceivedAndDeleted(bpRegistration, bpos.getBlockPoolId(),
report);
success = true;
} finally {
synchronized (pendingIncrementalBR) {
if (!success) {
// If we didn't succeed in sending the report, put all of the
// blocks back onto our queue, but only in the case where we didn't
// put something newer in the meantime.
for (ReceivedDeletedBlockInfo rdbi : receivedAndDeletedBlockArray) {
if (!pendingIncrementalBR.containsKey(rdbi.getBlock().getBlockId())) {
pendingIncrementalBR.put(rdbi.getBlock().getBlockId(), rdbi);
}
}
}
pendingReceivedRequests = pendingIncrementalBR.size();
}
}
}
}
/*
* Informing the name node could take a long long time! Should we wait
* till namenode is informed before responding with success to the
* client? For now we don't.
*/
void notifyNamenodeBlockImmediately(ReceivedDeletedBlockInfo bInfo) {
synchronized (pendingIncrementalBR) {
pendingIncrementalBR.put(
bInfo.getBlock().getBlockId(), bInfo);
pendingReceivedRequests++;
pendingIncrementalBR.notifyAll();
}
}
void notifyNamenodeDeletedBlock(ReceivedDeletedBlockInfo bInfo) {
synchronized (pendingIncrementalBR) {
pendingIncrementalBR.put(
bInfo.getBlock().getBlockId(), bInfo);
}
}
/**
* Run an immediate block report on this thread. Used by tests.
*/
@VisibleForTesting
void triggerBlockReportForTests() throws IOException {
synchronized (pendingIncrementalBR) {
lastBlockReport = 0;
lastHeartbeat = 0;
pendingIncrementalBR.notifyAll();
while (lastBlockReport == 0) {
try {
pendingIncrementalBR.wait(100);
} catch (InterruptedException e) {
return;
}
}
}
}
@VisibleForTesting
void triggerHeartbeatForTests() throws IOException {
synchronized (pendingIncrementalBR) {
lastHeartbeat = 0;
pendingIncrementalBR.notifyAll();
while (lastHeartbeat == 0) {
try {
pendingIncrementalBR.wait(100);
} catch (InterruptedException e) {
return;
}
}
}
}
@VisibleForTesting
void triggerDeletionReportForTests() throws IOException {
synchronized (pendingIncrementalBR) {
lastDeletedReport = 0;
pendingIncrementalBR.notifyAll();
while (lastDeletedReport == 0) {
try {
pendingIncrementalBR.wait(100);
} catch (InterruptedException e) {
return;
}
}
}
}
/**
* Report the list blocks to the Namenode
* @throws IOException
*/
DatanodeCommand blockReport() throws IOException {
// send block report if timer has expired.
DatanodeCommand cmd = null;
long startTime = now();
if (startTime - lastBlockReport > dnConf.blockReportInterval) {
// Flush any block information that precedes the block report. Otherwise
// we have a chance that we will miss the delHint information
// or we will report an RBW replica after the BlockReport already reports
// a FINALIZED one.
reportReceivedDeletedBlocks();
// Create block report
long brCreateStartTime = now();
BlockListAsLongs bReport = dn.getFSDataset().getBlockReport(
bpos.getBlockPoolId());
// Send block report
long brSendStartTime = now();
StorageBlockReport[] report = { new StorageBlockReport(
bpRegistration.getStorageID(), bReport.getBlockListAsLongs()) };
cmd = bpNamenode.blockReport(bpRegistration, bpos.getBlockPoolId(), report);
// Log the block report processing stats from Datanode perspective
long brSendCost = now() - brSendStartTime;
long brCreateCost = brSendStartTime - brCreateStartTime;
dn.getMetrics().addBlockReport(brSendCost);
LOG.info("BlockReport of " + bReport.getNumberOfBlocks()
+ " blocks took " + brCreateCost + " msec to generate and "
+ brSendCost + " msecs for RPC and NN processing");
// If we have sent the first block report, then wait a random
// time before we start the periodic block reports.
if (resetBlockReportTime) {
lastBlockReport = startTime - DFSUtil.getRandom().nextInt((int)(dnConf.blockReportInterval));
resetBlockReportTime = false;
} else {
/* say the last block report was at 8:20:14. The current report
* should have started around 9:20:14 (default 1 hour interval).
* If current time is :
* 1) normal like 9:20:18, next report should be at 10:20:14
* 2) unexpected like 11:35:43, next report should be at 12:20:14
*/
lastBlockReport += (now() - lastBlockReport) /
dnConf.blockReportInterval * dnConf.blockReportInterval;
}
LOG.info("sent block report, processed command:" + cmd);
}
return cmd;
}
HeartbeatResponse sendHeartBeat() throws IOException {
LOG.info("heartbeat: " + this);
// reports number of failed volumes
StorageReport[] report = { new StorageReport(bpRegistration.getStorageID(),
false,
dn.getFSDataset().getCapacity(),
dn.getFSDataset().getDfsUsed(),
dn.getFSDataset().getRemaining(),
dn.getFSDataset().getBlockPoolUsed(bpos.getBlockPoolId())) };
return bpNamenode.sendHeartbeat(bpRegistration, report,
dn.getXmitsInProgress(),
dn.getXceiverCount(),
dn.getFSDataset().getNumFailedVolumes());
}
//This must be called only by BPOfferService
void start() {
if ((bpThread != null) && (bpThread.isAlive())) {
//Thread is started already
return;
}
bpThread = new Thread(this, formatThreadName());
bpThread.setDaemon(true); // needed for JUnit testing
bpThread.start();
}
private String formatThreadName() {
Collection<URI> dataDirs = DataNode.getStorageDirs(dn.getConf());
return "DataNode: [" +
StringUtils.uriToString(dataDirs.toArray(new URI[0])) + "] " +
" heartbeating to " + nnAddr;
}
//This must be called only by blockPoolManager.
void stop() {
shouldServiceRun = false;
if (bpThread != null) {
bpThread.interrupt();
}
}
//This must be called only by blockPoolManager
void join() {
try {
if (bpThread != null) {
bpThread.join();
}
} catch (InterruptedException ie) { }
}
//Cleanup method to be called by current thread before exiting.
private synchronized void cleanUp() {
shouldServiceRun = false;
IOUtils.cleanup(LOG, bpNamenode);
bpos.shutdownActor(this);
}
/**
* Main loop for each BP thread. Run until shutdown,
* forever calling remote NameNode functions.
*/
private void offerService() throws Exception {
LOG.info("For namenode " + nnAddr + " using DELETEREPORT_INTERVAL of "
+ dnConf.deleteReportInterval + " msec " + " BLOCKREPORT_INTERVAL of "
+ dnConf.blockReportInterval + "msec" + " Initial delay: "
+ dnConf.initialBlockReportDelay + "msec" + "; heartBeatInterval="
+ dnConf.heartBeatInterval);
//
// Now loop for a long time....
//
while (shouldRun()) {
try {
long startTime = now();
//
// Every so often, send heartbeat or block-report
//
if (startTime - lastHeartbeat > dnConf.heartBeatInterval) {
//
// All heartbeat messages include following info:
// -- Datanode name
// -- data transfer port
// -- Total capacity
// -- Bytes remaining
//
lastHeartbeat = startTime;
if (!dn.areHeartbeatsDisabledForTests()) {
HeartbeatResponse resp = sendHeartBeat();
assert resp != null;
dn.getMetrics().addHeartbeat(now() - startTime);
// If the state of this NN has changed (eg STANDBY->ACTIVE)
// then let the BPOfferService update itself.
//
// Important that this happens before processCommand below,
// since the first heartbeat to a new active might have commands
// that we should actually process.
bpos.updateActorStatesFromHeartbeat(
this, resp.getNameNodeHaState());
long startProcessCommands = now();
if (!processCommand(resp.getCommands()))
continue;
long endProcessCommands = now();
if (endProcessCommands - startProcessCommands > 2000) {
LOG.info("Took " + (endProcessCommands - startProcessCommands)
+ "ms to process " + resp.getCommands().length
+ " commands from NN");
}
}
}
if (pendingReceivedRequests > 0
|| (startTime - lastDeletedReport > dnConf.deleteReportInterval)) {
reportReceivedDeletedBlocks();
lastDeletedReport = startTime;
}
DatanodeCommand cmd = blockReport();
processCommand(new DatanodeCommand[]{ cmd });
// Now safe to start scanning the block pool.
// If it has already been started, this is a no-op.
if (dn.blockScanner != null) {
dn.blockScanner.addBlockPool(bpos.getBlockPoolId());
}
//
// There is no work to do; sleep until hearbeat timer elapses,
// or work arrives, and then iterate again.
//
long waitTime = dnConf.heartBeatInterval -
(System.currentTimeMillis() - lastHeartbeat);
synchronized(pendingIncrementalBR) {
if (waitTime > 0 && pendingReceivedRequests == 0) {
try {
pendingIncrementalBR.wait(waitTime);
} catch (InterruptedException ie) {
LOG.warn("BPOfferService for " + this + " interrupted");
}
}
} // synchronized
} catch(RemoteException re) {
String reClass = re.getClassName();
if (UnregisteredNodeException.class.getName().equals(reClass) ||
DisallowedDatanodeException.class.getName().equals(reClass) ||
IncorrectVersionException.class.getName().equals(reClass)) {
LOG.warn(this + " is shutting down", re);
shouldServiceRun = false;
return;
}
LOG.warn("RemoteException in offerService", re);
try {
long sleepTime = Math.min(1000, dnConf.heartBeatInterval);
Thread.sleep(sleepTime);
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
}
} catch (IOException e) {
LOG.warn("IOException in offerService", e);
}
} // while (shouldRun())
} // offerService
/**
* Register one bp with the corresponding NameNode
* <p>
* The bpDatanode needs to register with the namenode on startup in order
* 1) to report which storage it is serving now and
* 2) to receive a registrationID
*
* issued by the namenode to recognize registered datanodes.
*
* @see FSNamesystem#registerDatanode(DatanodeRegistration)
* @throws IOException
*/
void register() throws IOException {
// The handshake() phase loaded the block pool storage
// off disk - so update the bpRegistration object from that info
bpRegistration = bpos.createRegistration();
LOG.info(this + " beginning handshake with NN");
while (shouldRun()) {
try {
// Use returned registration from namenode with updated machine name.
bpRegistration = bpNamenode.registerDatanode(bpRegistration,
new DatanodeStorage[0]);
break;
} catch(SocketTimeoutException e) { // namenode is busy
LOG.info("Problem connecting to server: " + nnAddr);
sleepAndLogInterrupts(1000, "connecting to server");
}
}
LOG.info("Block pool " + this + " successfully registered with NN");
bpos.registrationSucceeded(this, bpRegistration);
// random short delay - helps scatter the BR from all DNs
scheduleBlockReport(dnConf.initialBlockReportDelay);
}
private void sleepAndLogInterrupts(int millis,
String stateString) {
try {
Thread.sleep(millis);
} catch (InterruptedException ie) {
LOG.info("BPOfferService " + this +
" interrupted while " + stateString);
}
}
/**
* No matter what kind of exception we get, keep retrying to offerService().
* That's the loop that connects to the NameNode and provides basic DataNode
* functionality.
*
* Only stop when "shouldRun" or "shouldServiceRun" is turned off, which can
* happen either at shutdown or due to refreshNamenodes.
*/
@Override
public void run() {
LOG.info(this + " starting to offer service");
try {
// init stuff
try {
// setup storage
connectToNNAndHandshake();
} catch (IOException ioe) {
// Initial handshake, storage recovery or registration failed
// End BPOfferService thread
LOG.fatal("Initialization failed for block pool " + this, ioe);
return;
}
initialized = true; // bp is initialized;
while (shouldRun()) {
try {
bpos.startDistributedUpgradeIfNeeded();
offerService();
} catch (Exception ex) {
LOG.error("Exception in BPOfferService for " + this, ex);
sleepAndLogInterrupts(5000, "offering service");
}
}
} catch (Throwable ex) {
LOG.warn("Unexpected exception in block pool " + this, ex);
} finally {
LOG.warn("Ending block pool service for: " + this);
cleanUp();
}
}
private boolean shouldRun() {
return shouldServiceRun && dn.shouldRun();
}
/**
* Process an array of datanode commands
*
* @param cmds an array of datanode commands
* @return true if further processing may be required or false otherwise.
*/
boolean processCommand(DatanodeCommand[] cmds) {
if (cmds != null) {
for (DatanodeCommand cmd : cmds) {
try {
if (bpos.processCommandFromActor(cmd, this) == false) {
return false;
}
} catch (IOException ioe) {
LOG.warn("Error processing datanode Command", ioe);
}
}
}
return true;
}
void trySendErrorReport(int errCode, String errMsg) {
try {
bpNamenode.errorReport(bpRegistration, errCode, errMsg);
} catch(IOException e) {
LOG.warn("Error reporting an error to NameNode " + nnAddr,
e);
}
}
/**
* Report a bad block from another DN in this cluster.
*/
void reportRemoteBadBlock(DatanodeInfo dnInfo, ExtendedBlock block)
throws IOException {
LocatedBlock lb = new LocatedBlock(block,
new DatanodeInfo[] {dnInfo});
bpNamenode.reportBadBlocks(new LocatedBlock[] {lb});
}
void reRegister() throws IOException {
if (shouldRun()) {
// re-retrieve namespace info to make sure that, if the NN
// was restarted, we still match its version (HDFS-2120)
retrieveNamespaceInfo();
// and re-register
register();
}
}
}

View File

@ -0,0 +1,241 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.datanode;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.commons.logging.Log;
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.security.UserGroupInformation;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
/**
* Manages the BPOfferService objects for the data node.
* Creation, removal, starting, stopping, shutdown on BPOfferService
* objects must be done via APIs in this class.
*/
@InterfaceAudience.Private
class BlockPoolManager {
private static final Log LOG = DataNode.LOG;
private final Map<String, BPOfferService> bpByNameserviceId =
Maps.newHashMap();
private final Map<String, BPOfferService> bpByBlockPoolId =
Maps.newHashMap();
private final List<BPOfferService> offerServices =
Lists.newArrayList();
private final DataNode dn;
//This lock is used only to ensure exclusion of refreshNamenodes
private final Object refreshNamenodesLock = new Object();
BlockPoolManager(DataNode dn) {
this.dn = dn;
}
synchronized void addBlockPool(BPOfferService bpos) {
Preconditions.checkArgument(offerServices.contains(bpos),
"Unknown BPOS: %s", bpos);
if (bpos.getBlockPoolId() == null) {
throw new IllegalArgumentException("Null blockpool id");
}
bpByBlockPoolId.put(bpos.getBlockPoolId(), bpos);
}
/**
* Returns the array of BPOfferService objects.
* Caution: The BPOfferService returned could be shutdown any time.
*/
synchronized BPOfferService[] getAllNamenodeThreads() {
BPOfferService[] bposArray = new BPOfferService[offerServices.size()];
return offerServices.toArray(bposArray);
}
synchronized BPOfferService get(String bpid) {
return bpByBlockPoolId.get(bpid);
}
synchronized void remove(BPOfferService t) {
offerServices.remove(t);
bpByBlockPoolId.remove(t.getBlockPoolId());
boolean removed = false;
for (Iterator<BPOfferService> it = bpByNameserviceId.values().iterator();
it.hasNext() && !removed;) {
BPOfferService bpos = it.next();
if (bpos == t) {
it.remove();
LOG.info("Removed " + bpos);
removed = true;
}
}
if (!removed) {
LOG.warn("Couldn't remove BPOS " + t + " from bpByNameserviceId map");
}
}
void shutDownAll() throws InterruptedException {
BPOfferService[] bposArray = this.getAllNamenodeThreads();
for (BPOfferService bpos : bposArray) {
bpos.stop(); //interrupts the threads
}
//now join
for (BPOfferService bpos : bposArray) {
bpos.join();
}
}
synchronized void startAll() throws IOException {
try {
UserGroupInformation.getLoginUser().doAs(
new PrivilegedExceptionAction<Object>() {
public Object run() throws Exception {
for (BPOfferService bpos : offerServices) {
bpos.start();
}
return null;
}
});
} catch (InterruptedException ex) {
IOException ioe = new IOException();
ioe.initCause(ex.getCause());
throw ioe;
}
}
void joinAll() {
for (BPOfferService bpos: this.getAllNamenodeThreads()) {
bpos.join();
}
}
void refreshNamenodes(Configuration conf)
throws IOException {
LOG.info("Refresh request received for nameservices: "
+ conf.get(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES));
Map<String, Map<String, InetSocketAddress>> newAddressMap =
DFSUtil.getNNServiceRpcAddresses(conf);
synchronized (refreshNamenodesLock) {
doRefreshNamenodes(newAddressMap);
}
}
private void doRefreshNamenodes(
Map<String, Map<String, InetSocketAddress>> addrMap) throws IOException {
assert Thread.holdsLock(refreshNamenodesLock);
Set<String> toRefresh = Sets.newHashSet();
Set<String> toAdd = Sets.newHashSet();
Set<String> toRemove;
synchronized (this) {
// Step 1. For each of the new nameservices, figure out whether
// it's an update of the set of NNs for an existing NS,
// or an entirely new nameservice.
for (String nameserviceId : addrMap.keySet()) {
if (bpByNameserviceId.containsKey(nameserviceId)) {
toRefresh.add(nameserviceId);
} else {
toAdd.add(nameserviceId);
}
}
// Step 2. Any nameservices we currently have but are no longer present
// need to be removed.
toRemove = Sets.newHashSet(Sets.difference(
bpByNameserviceId.keySet(), addrMap.keySet()));
assert toRefresh.size() + toAdd.size() ==
addrMap.size() :
"toAdd: " + Joiner.on(",").useForNull("<default>").join(toAdd) +
" toRemove: " + Joiner.on(",").useForNull("<default>").join(toRemove) +
" toRefresh: " + Joiner.on(",").useForNull("<default>").join(toRefresh);
// Step 3. Start new nameservices
if (!toAdd.isEmpty()) {
LOG.info("Starting BPOfferServices for nameservices: " +
Joiner.on(",").useForNull("<default>").join(toAdd));
for (String nsToAdd : toAdd) {
ArrayList<InetSocketAddress> addrs =
Lists.newArrayList(addrMap.get(nsToAdd).values());
BPOfferService bpos = createBPOS(addrs);
bpByNameserviceId.put(nsToAdd, bpos);
offerServices.add(bpos);
}
}
startAll();
}
// Step 4. Shut down old nameservices. This happens outside
// of the synchronized(this) lock since they need to call
// back to .remove() from another thread
if (!toRemove.isEmpty()) {
LOG.info("Stopping BPOfferServices for nameservices: " +
Joiner.on(",").useForNull("<default>").join(toRemove));
for (String nsToRemove : toRemove) {
BPOfferService bpos = bpByNameserviceId.get(nsToRemove);
bpos.stop();
bpos.join();
// they will call remove on their own
}
}
// Step 5. Update nameservices whose NN list has changed
if (!toRefresh.isEmpty()) {
LOG.info("Refreshing list of NNs for nameservices: " +
Joiner.on(",").useForNull("<default>").join(toRefresh));
for (String nsToRefresh : toRefresh) {
BPOfferService bpos = bpByNameserviceId.get(nsToRefresh);
ArrayList<InetSocketAddress> addrs =
Lists.newArrayList(addrMap.get(nsToRefresh).values());
bpos.refreshNNList(addrs);
}
}
}
/**
* Extracted out for test purposes.
*/
protected BPOfferService createBPOS(List<InetSocketAddress> nnAddrs) {
return new BPOfferService(nnAddrs, dn);
}
}

View File

@ -153,6 +153,7 @@ class BlockReceiver implements Closeable {
switch (stage) {
case PIPELINE_SETUP_CREATE:
replicaInfo = datanode.data.createRbw(block);
datanode.notifyNamenodeReceivingBlock(block);
break;
case PIPELINE_SETUP_STREAMING_RECOVERY:
replicaInfo = datanode.data.recoverRbw(
@ -166,6 +167,7 @@ class BlockReceiver implements Closeable {
block.getLocalBlock());
}
block.setGenerationStamp(newGs);
datanode.notifyNamenodeReceivingBlock(block);
break;
case PIPELINE_SETUP_APPEND_RECOVERY:
replicaInfo = datanode.data.recoverAppend(block, newGs, minBytesRcvd);
@ -174,6 +176,7 @@ class BlockReceiver implements Closeable {
block.getLocalBlock());
}
block.setGenerationStamp(newGs);
datanode.notifyNamenodeReceivingBlock(block);
break;
case TRANSFER_RBW:
case TRANSFER_FINALIZED:
@ -320,7 +323,6 @@ class BlockReceiver implements Closeable {
private void verifyChunks( byte[] dataBuf, int dataOff, int len,
byte[] checksumBuf, int checksumOff )
throws IOException {
DatanodeProtocol nn = datanode.getBPNamenode(block.getBlockPoolId());
while (len > 0) {
int chunkLen = Math.min(len, bytesPerChecksum);
@ -331,9 +333,7 @@ class BlockReceiver implements Closeable {
try {
LOG.info("report corrupt block " + block + " from datanode " +
srcDataNode + " to namenode");
LocatedBlock lb = new LocatedBlock(block,
new DatanodeInfo[] {srcDataNode});
nn.reportBadBlocks(new LocatedBlock[] {lb});
datanode.reportRemoteBadBlock(srcDataNode, block);
} catch (IOException e) {
LOG.warn("Failed to report bad block " + block +
" from datanode " + srcDataNode + " to namenode");

View File

@ -45,7 +45,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOUR
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STARTUP_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_FEDERATION_NAMESERVICES;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTPS_ENABLE_KEY;
import java.io.BufferedOutputStream;
@ -86,6 +85,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
import org.apache.hadoop.hdfs.HDFSPolicyProvider;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.Block;
@ -164,6 +164,8 @@ import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.VersionInfo;
import org.mortbay.util.ajax.JSON;
import com.google.common.base.Preconditions;
import com.google.common.collect.Sets;
import com.google.protobuf.BlockingService;
@ -230,143 +232,6 @@ public class DataNode extends Configured
return NetUtils.createSocketAddr(target);
}
/**
* Manages he BPOfferService objects for the data node.
* Creation, removal, starting, stopping, shutdown on BPOfferService
* objects must be done via APIs in this class.
*/
@InterfaceAudience.Private
class BlockPoolManager {
private final Map<String, BPOfferService> bpMapping;
private final Map<InetSocketAddress, BPOfferService> nameNodeThreads;
//This lock is used only to ensure exclusion of refreshNamenodes
private final Object refreshNamenodesLock = new Object();
BlockPoolManager(Configuration conf)
throws IOException {
bpMapping = new HashMap<String, BPOfferService>();
nameNodeThreads = new HashMap<InetSocketAddress, BPOfferService>();
List<InetSocketAddress> isas = DFSUtil.getNNServiceRpcAddresses(conf);
for(InetSocketAddress isa : isas) {
BPOfferService bpos = new BPOfferService(isa, DataNode.this);
nameNodeThreads.put(bpos.getNNSocketAddress(), bpos);
}
}
synchronized void addBlockPool(BPOfferService t) {
if (nameNodeThreads.get(t.getNNSocketAddress()) == null) {
throw new IllegalArgumentException(
"Unknown BPOfferService thread for namenode address:"
+ t.getNNSocketAddress());
}
if (t.getBlockPoolId() == null) {
throw new IllegalArgumentException("Null blockpool id");
}
bpMapping.put(t.getBlockPoolId(), t);
}
/**
* Returns the array of BPOfferService objects.
* Caution: The BPOfferService returned could be shutdown any time.
*/
synchronized BPOfferService[] getAllNamenodeThreads() {
BPOfferService[] bposArray = new BPOfferService[nameNodeThreads.values()
.size()];
return nameNodeThreads.values().toArray(bposArray);
}
synchronized BPOfferService get(InetSocketAddress addr) {
return nameNodeThreads.get(addr);
}
synchronized BPOfferService get(String bpid) {
return bpMapping.get(bpid);
}
synchronized void remove(BPOfferService t) {
nameNodeThreads.remove(t.getNNSocketAddress());
bpMapping.remove(t.getBlockPoolId());
}
void shutDownAll() throws InterruptedException {
BPOfferService[] bposArray = this.getAllNamenodeThreads();
for (BPOfferService bpos : bposArray) {
bpos.stop(); //interrupts the threads
}
//now join
for (BPOfferService bpos : bposArray) {
bpos.join();
}
}
synchronized void startAll() throws IOException {
try {
UserGroupInformation.getLoginUser().doAs(
new PrivilegedExceptionAction<Object>() {
public Object run() throws Exception {
for (BPOfferService bpos : nameNodeThreads.values()) {
bpos.start();
}
return null;
}
});
} catch (InterruptedException ex) {
IOException ioe = new IOException();
ioe.initCause(ex.getCause());
throw ioe;
}
}
void joinAll() {
for (BPOfferService bpos: this.getAllNamenodeThreads()) {
bpos.join();
}
}
void refreshNamenodes(Configuration conf)
throws IOException {
LOG.info("Refresh request received for nameservices: "
+ conf.get(DFS_FEDERATION_NAMESERVICES));
List<InetSocketAddress> newAddresses =
DFSUtil.getNNServiceRpcAddresses(conf);
List<BPOfferService> toShutdown = new ArrayList<BPOfferService>();
List<InetSocketAddress> toStart = new ArrayList<InetSocketAddress>();
synchronized (refreshNamenodesLock) {
synchronized (this) {
for (InetSocketAddress nnaddr : nameNodeThreads.keySet()) {
if (!(newAddresses.contains(nnaddr))) {
toShutdown.add(nameNodeThreads.get(nnaddr));
}
}
for (InetSocketAddress nnaddr : newAddresses) {
if (!(nameNodeThreads.containsKey(nnaddr))) {
toStart.add(nnaddr);
}
}
for (InetSocketAddress nnaddr : toStart) {
BPOfferService bpos = new BPOfferService(nnaddr, DataNode.this);
nameNodeThreads.put(bpos.getNNSocketAddress(), bpos);
}
}
for (BPOfferService bpos : toShutdown) {
bpos.stop();
bpos.join();
}
// stoping the BPOSes causes them to call remove() on their own when they
// clean up.
// Now start the threads that are not already running.
startAll();
}
}
}
volatile boolean shouldRun = true;
private BlockPoolManager blockPoolManager;
public volatile FSDatasetInterface<? extends FSVolumeInterface> data = null;
@ -653,7 +518,18 @@ public class DataNode extends Configured
if(bpos != null) {
bpos.notifyNamenodeReceivedBlock(block, delHint);
} else {
LOG.warn("Cannot find BPOfferService for reporting block received for bpid="
LOG.error("Cannot find BPOfferService for reporting block received for bpid="
+ block.getBlockPoolId());
}
}
// calls specific to BP
protected void notifyNamenodeReceivingBlock(ExtendedBlock block) {
BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
if(bpos != null) {
bpos.notifyNamenodeReceivingBlock(block);
} else {
LOG.error("Cannot find BPOfferService for reporting block receiving for bpid="
+ block.getBlockPoolId());
}
}
@ -664,18 +540,66 @@ public class DataNode extends Configured
if (bpos != null) {
bpos.notifyNamenodeDeletedBlock(block);
} else {
LOG.warn("Cannot find BPOfferService for reporting block deleted for bpid="
LOG.error("Cannot find BPOfferService for reporting block deleted for bpid="
+ block.getBlockPoolId());
}
}
/**
* Report a bad block which is hosted on the local DN.
*/
public void reportBadBlocks(ExtendedBlock block) throws IOException{
BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
if(bpos == null || bpos.bpNamenode == null) {
throw new IOException("cannot locate OfferService thread for bp="+block.getBlockPoolId());
}
BPOfferService bpos = getBPOSForBlock(block);
bpos.reportBadBlocks(block);
}
/**
* Report a bad block on another DN (eg if we received a corrupt replica
* from a remote host).
* @param srcDataNode the DN hosting the bad block
* @param block the block itself
*/
public void reportRemoteBadBlock(DatanodeInfo srcDataNode, ExtendedBlock block)
throws IOException {
BPOfferService bpos = getBPOSForBlock(block);
bpos.reportRemoteBadBlock(srcDataNode, block);
}
/**
* Try to send an error report to the NNs associated with the given
* block pool.
* @param bpid the block pool ID
* @param errCode error code to send
* @param errMsg textual message to send
*/
void trySendErrorReport(String bpid, int errCode, String errMsg) {
BPOfferService bpos = blockPoolManager.get(bpid);
if (bpos == null) {
throw new IllegalArgumentException("Bad block pool: " + bpid);
}
bpos.trySendErrorReport(errCode, errMsg);
}
/**
* Return the BPOfferService instance corresponding to the given block.
* @param block
* @return the BPOS
* @throws IOException if no such BPOS can be found
*/
private BPOfferService getBPOSForBlock(ExtendedBlock block)
throws IOException {
Preconditions.checkNotNull(block);
BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
if (bpos == null) {
throw new IOException("cannot locate OfferService thread for bp="+
block.getBlockPoolId());
}
return bpos;
}
// used only for testing
void setHeartbeatsDisabledForTests(
@ -728,7 +652,8 @@ public class DataNode extends Configured
metrics = DataNodeMetrics.create(conf, getMachineName());
blockPoolManager = new BlockPoolManager(conf);
blockPoolManager = new BlockPoolManager(this);
blockPoolManager.refreshNamenodes(conf);
}
/**
@ -961,11 +886,15 @@ public class DataNode extends Configured
/**
* get BP registration by machine and port name (host:port)
* @param mName
* @param mName - the name that the NN used
* @return BP registration
* @throws IOException
*/
DatanodeRegistration getDNRegistrationByMachineName(String mName) {
// TODO: all the BPs should have the same name as each other, they all come
// from getName() here! and the use cases only are in tests where they just
// call with getName(). So we could probably just make this method return
// the first BPOS's registration. See HDFS-2609.
BPOfferService [] bposArray = blockPoolManager.getAllNamenodeThreads();
for (BPOfferService bpos : bposArray) {
if(bpos.bpRegistration.getName().equals(mName))
@ -1011,20 +940,6 @@ public class DataNode extends Configured
throw new IOException(ie.getMessage());
}
}
/**
* get the name node address based on the block pool id
* @param bpid block pool ID
* @return namenode address corresponding to the bpid
*/
public InetSocketAddress getNameNodeAddr(String bpid) {
BPOfferService bp = blockPoolManager.get(bpid);
if (bp != null) {
return bp.getNNSocketAddress();
}
LOG.warn("No name node address found for block pool ID " + bpid);
return null;
}
public InetSocketAddress getSelfAddr() {
return selfAddr;
@ -1251,12 +1166,7 @@ public class DataNode extends Configured
//inform NameNodes
for(BPOfferService bpos: blockPoolManager.getAllNamenodeThreads()) {
DatanodeProtocolClientSideTranslatorPB nn = bpos.bpNamenode;
try {
nn.errorReport(bpos.bpRegistration, dpError, errMsgr);
} catch(IOException e) {
LOG.warn("Error reporting disk failure to NameNode", e);
}
bpos.trySendErrorReport(dpError, errMsgr);
}
if(hasEnoughResources) {
@ -1273,6 +1183,10 @@ public class DataNode extends Configured
public int getXceiverCount() {
return threadGroup == null ? 0 : threadGroup.activeCount();
}
int getXmitsInProgress() {
return xmitsInProgress.get();
}
UpgradeManagerDatanode getUpgradeManagerDatanode(String bpid) {
BPOfferService bpos = blockPoolManager.get(bpid);
@ -1285,15 +1199,15 @@ public class DataNode extends Configured
private void transferBlock( ExtendedBlock block,
DatanodeInfo xferTargets[]
) throws IOException {
DatanodeProtocolClientSideTranslatorPB nn = getBPNamenode(block
.getBlockPoolId());
BPOfferService bpos = getBPOSForBlock(block);
DatanodeRegistration bpReg = getDNRegistrationForBP(block.getBlockPoolId());
if (!data.isValidBlock(block)) {
// block does not exist or is under-construction
String errStr = "Can't send invalid block " + block;
LOG.info(errStr);
nn.errorReport(bpReg, DatanodeProtocol.INVALID_BLOCK, errStr);
bpos.trySendErrorReport(DatanodeProtocol.INVALID_BLOCK, errStr);
return;
}
@ -1301,9 +1215,7 @@ public class DataNode extends Configured
long onDiskLength = data.getLength(block);
if (block.getNumBytes() > onDiskLength) {
// Shorter on-disk len indicates corruption so report NN the corrupt block
nn.reportBadBlocks(new LocatedBlock[]{
new LocatedBlock(block, new DatanodeInfo[] {
new DatanodeInfo(bpReg)})});
bpos.reportBadBlocks(block);
LOG.warn("Can't replicate block " + block
+ " because on-disk length " + onDiskLength
+ " is shorter than NameNode recorded length " + block.getNumBytes());
@ -1861,6 +1773,13 @@ public class DataNode extends Configured
long newLength) throws IOException {
ReplicaInfo r = data.updateReplicaUnderRecovery(oldBlock,
recoveryId, newLength);
// Notify the namenode of the updated block info. This is important
// for HA, since otherwise the standby node may lose track of the
// block locations until the next block report.
ExtendedBlock newBlock = new ExtendedBlock(oldBlock);
newBlock.setGenerationStamp(recoveryId);
newBlock.setNumBytes(newLength);
notifyNamenodeReceivedBlock(newBlock, "");
return new ExtendedBlock(oldBlock.getBlockPoolId(), r);
}
@ -1935,23 +1854,32 @@ public class DataNode extends Configured
* @return Namenode corresponding to the bpid
* @throws IOException
*/
public DatanodeProtocolClientSideTranslatorPB getBPNamenode(String bpid)
public DatanodeProtocolClientSideTranslatorPB getActiveNamenodeForBP(String bpid)
throws IOException {
BPOfferService bpos = blockPoolManager.get(bpid);
if (bpos == null) {
throw new IOException("No block pool offer service for bpid=" + bpid);
} else if (bpos.bpNamenode == null) {
throw new IOException("cannot find a namenode proxy for bpid=" + bpid);
}
return bpos.bpNamenode;
DatanodeProtocolClientSideTranslatorPB activeNN = bpos.getActiveNN();
if (activeNN == null) {
throw new IOException(
"Block pool " + bpid + " has not recognized an active NN");
}
return activeNN;
}
/** Block synchronization */
void syncBlock(RecoveringBlock rBlock,
List<BlockRecord> syncList) throws IOException {
ExtendedBlock block = rBlock.getBlock();
DatanodeProtocolClientSideTranslatorPB nn = getBPNamenode(block
.getBlockPoolId());
DatanodeProtocolClientSideTranslatorPB nn =
getActiveNamenodeForBP(block.getBlockPoolId());
if (nn == null) {
throw new IOException(
"Unable to synchronize block " + rBlock + ", since this DN "
+ " has not acknowledged any NN as active.");
}
long recoveryId = rBlock.getNewGenerationStamp();
if (LOG.isDebugEnabled()) {
@ -2172,14 +2100,19 @@ public class DataNode extends Configured
/**
* Returned information is a JSON representation of a map with
* name node host name as the key and block pool Id as the value
* name node host name as the key and block pool Id as the value.
* Note that, if there are multiple NNs in an NA nameservice,
* a given block pool may be represented twice.
*/
@Override // DataNodeMXBean
public String getNamenodeAddresses() {
final Map<String, String> info = new HashMap<String, String>();
for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) {
if (bpos != null && bpos.bpThread != null) {
info.put(bpos.getNNSocketAddress().getHostName(), bpos.getBlockPoolId());
if (bpos != null) {
for (BPServiceActor actor : bpos.getBPServiceActors()) {
info.put(actor.getNNSocketAddress().getHostName(),
bpos.getBlockPoolId());
}
}
}
return JSON.toString(info);
@ -2228,11 +2161,18 @@ public class DataNode extends Configured
/**
* @param addr rpc address of the namenode
* @return true - if BPOfferService corresponding to the namenode is alive
* @return true if the datanode is connected to a NameNode at the
* given address
*/
public boolean isBPServiceAlive(InetSocketAddress addr) {
BPOfferService bp = blockPoolManager.get(addr);
return bp != null ? bp.isAlive() : false;
public boolean isConnectedToNN(InetSocketAddress addr) {
for (BPOfferService bpos : getAllBpOs()) {
for (BPServiceActor bpsa : bpos.getBPServiceActors()) {
if (addr.equals(bpsa.getNNSocketAddress())) {
return bpsa.isAlive();
}
}
}
return false;
}
/**

Some files were not shown because too many files have changed in this diff Show More