HDFS-6475. WebHdfs clients fail without retry because incorrect handling of StandbyException. Contributed by Yongjun Zhang.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1605217 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Aaron Myers 2014-06-24 23:07:49 +00:00
parent e285b98f0f
commit 55a1defa50
3 changed files with 124 additions and 3 deletions

View File

@ -716,6 +716,9 @@ Release 2.5.0 - UNRELEASED
HDFS-6598. Fix a typo in message issued from explorer.js. (Yongjun Zhang HDFS-6598. Fix a typo in message issued from explorer.js. (Yongjun Zhang
via wheat9) via wheat9)
HDFS-6475. WebHdfs clients fail without retry because incorrect handling
of StandbyException. (Yongjun Zhang via atm)
BREAKDOWN OF HDFS-2006 SUBTASKS AND RELATED JIRAS BREAKDOWN OF HDFS-2006 SUBTASKS AND RELATED JIRAS
HDFS-6299. Protobuf for XAttr and client-side implementation. (Yi Liu via umamahesh) HDFS-6299. Protobuf for XAttr and client-side implementation. (Yi Liu via umamahesh)

View File

@ -31,8 +31,11 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.web.JsonUtil; import org.apache.hadoop.hdfs.web.JsonUtil;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.StandbyException;
import org.apache.hadoop.security.authorize.AuthorizationException; import org.apache.hadoop.security.authorize.AuthorizationException;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import com.google.common.annotations.VisibleForTesting;
import com.sun.jersey.api.ParamException; import com.sun.jersey.api.ParamException;
import com.sun.jersey.api.container.ContainerException; import com.sun.jersey.api.container.ContainerException;
@ -43,8 +46,21 @@ public class ExceptionHandler implements ExceptionMapper<Exception> {
private static Exception toCause(Exception e) { private static Exception toCause(Exception e) {
final Throwable t = e.getCause(); final Throwable t = e.getCause();
if (t != null && t instanceof Exception) { if (e instanceof SecurityException) {
e = (Exception)e.getCause(); // For the issue reported in HDFS-6475, if SecurityException's cause
// is InvalidToken, and the InvalidToken's cause is StandbyException,
// return StandbyException; Otherwise, leave the exception as is,
// since they are handled elsewhere. See HDFS-6588.
if (t != null && t instanceof InvalidToken) {
final Throwable t1 = t.getCause();
if (t1 != null && t1 instanceof StandbyException) {
e = (StandbyException)t1;
}
}
} else {
if (t != null && t instanceof Exception) {
e = (Exception)t;
}
} }
return e; return e;
} }
@ -74,6 +90,10 @@ public class ExceptionHandler implements ExceptionMapper<Exception> {
e = ((RemoteException)e).unwrapRemoteException(); e = ((RemoteException)e).unwrapRemoteException();
} }
if (e instanceof SecurityException) {
e = toCause(e);
}
//Map response status //Map response status
final Response.Status s; final Response.Status s;
if (e instanceof SecurityException) { if (e instanceof SecurityException) {
@ -96,4 +116,9 @@ public class ExceptionHandler implements ExceptionMapper<Exception> {
final String js = JsonUtil.toJsonString(e); final String js = JsonUtil.toJsonString(e);
return Response.status(s).type(MediaType.APPLICATION_JSON).entity(js).build(); return Response.status(s).type(MediaType.APPLICATION_JSON).entity(js).build();
} }
@VisibleForTesting
public void initResponse(HttpServletResponse response) {
this.response = response;
}
} }

View File

@ -23,6 +23,7 @@ import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull; import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import java.io.ByteArrayInputStream; import java.io.ByteArrayInputStream;
import java.io.DataInputStream; import java.io.DataInputStream;
@ -32,6 +33,10 @@ import java.net.URI;
import java.security.PrivilegedExceptionAction; import java.security.PrivilegedExceptionAction;
import java.util.Collection; import java.util.Collection;
import java.util.HashSet; import java.util.HashSet;
import java.util.Map;
import javax.servlet.http.HttpServletResponse;
import javax.ws.rs.core.Response;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -51,7 +56,10 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.hdfs.web.JsonUtil;
import org.apache.hadoop.hdfs.web.resources.ExceptionHandler;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.RetriableException; import org.apache.hadoop.ipc.RetriableException;
import org.apache.hadoop.ipc.StandbyException; import org.apache.hadoop.ipc.StandbyException;
import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.SecurityUtil;
@ -64,6 +72,7 @@ import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.mockito.internal.util.reflection.Whitebox; import org.mockito.internal.util.reflection.Whitebox;
import org.mortbay.util.ajax.JSON;
import com.google.common.base.Joiner; import com.google.common.base.Joiner;
@ -372,6 +381,90 @@ public class TestDelegationTokensWithHA {
token.cancel(conf); token.cancel(conf);
} }
/**
* Test if StandbyException can be thrown from StandbyNN, when it's requested for
* password. (HDFS-6475). With StandbyException, the client can failover to try
* activeNN.
*/
@Test
public void testDelegationTokenStandbyNNAppearFirst() throws Exception {
// make nn0 the standby NN, and nn1 the active NN
cluster.transitionToStandby(0);
cluster.transitionToActive(1);
final DelegationTokenSecretManager stSecretManager =
NameNodeAdapter.getDtSecretManager(
nn1.getNamesystem());
// create token
final Token<DelegationTokenIdentifier> token =
getDelegationToken(fs, "JobTracker");
final DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
byte[] tokenId = token.getIdentifier();
identifier.readFields(new DataInputStream(
new ByteArrayInputStream(tokenId)));
assertTrue(null != stSecretManager.retrievePassword(identifier));
final UserGroupInformation ugi = UserGroupInformation
.createRemoteUser("JobTracker");
ugi.addToken(token);
ugi.doAs(new PrivilegedExceptionAction<Object>() {
@Override
public Object run() {
try {
try {
byte[] tmppw = dtSecretManager.retrievePassword(identifier);
fail("InvalidToken with cause StandbyException is expected"
+ " since nn0 is standby");
return tmppw;
} catch (IOException e) {
// Mimic the UserProvider class logic (server side) by throwing
// SecurityException here
throw new SecurityException(
"Failed to obtain user group information: " + e, e);
}
} catch (Exception oe) {
//
// The exception oe caught here is
// java.lang.SecurityException: Failed to obtain user group
// information: org.apache.hadoop.security.token.
// SecretManager$InvalidToken: StandbyException
//
HttpServletResponse response = mock(HttpServletResponse.class);
ExceptionHandler eh = new ExceptionHandler();
eh.initResponse(response);
// The Response (resp) below is what the server will send to client
//
// BEFORE HDFS-6475 fix, the resp.entity is
// {"RemoteException":{"exception":"SecurityException",
// "javaClassName":"java.lang.SecurityException",
// "message":"Failed to obtain user group information:
// org.apache.hadoop.security.token.SecretManager$InvalidToken:
// StandbyException"}}
// AFTER the fix, the resp.entity is
// {"RemoteException":{"exception":"StandbyException",
// "javaClassName":"org.apache.hadoop.ipc.StandbyException",
// "message":"Operation category READ is not supported in
// state standby"}}
//
Response resp = eh.toResponse(oe);
// Mimic the client side logic by parsing the response from server
//
Map<?, ?> m = (Map<?, ?>)JSON.parse(resp.getEntity().toString());
RemoteException re = JsonUtil.toRemoteException(m);
Exception unwrapped = ((RemoteException)re).unwrapRemoteException(
StandbyException.class);
assertTrue (unwrapped instanceof StandbyException);
return null;
}
}
});
}
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
private Token<DelegationTokenIdentifier> getDelegationToken(FileSystem fs, private Token<DelegationTokenIdentifier> getDelegationToken(FileSystem fs,
String renewer) throws IOException { String renewer) throws IOException {