|
@@ -25,10 +25,13 @@ import java.io.PrintWriter;
|
|
|
import java.net.InetAddress;
|
|
|
import java.net.URI;
|
|
|
import java.net.URISyntaxException;
|
|
|
+import java.net.UnknownHostException;
|
|
|
+import java.security.Principal;
|
|
|
import java.security.PrivilegedExceptionAction;
|
|
|
import java.util.EnumSet;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.List;
|
|
|
+import java.util.concurrent.ExecutionException;
|
|
|
|
|
|
import javax.servlet.ServletContext;
|
|
|
import javax.servlet.http.HttpServletRequest;
|
|
@@ -58,6 +61,7 @@ import org.apache.hadoop.fs.Options;
|
|
|
import org.apache.hadoop.fs.XAttr;
|
|
|
import org.apache.hadoop.fs.permission.AclStatus;
|
|
|
import org.apache.hadoop.fs.permission.FsAction;
|
|
|
+import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.hdfs.XAttrHelper;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
@@ -79,8 +83,8 @@ import org.apache.hadoop.hdfs.web.WebHdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
|
|
|
import org.apache.hadoop.hdfs.web.resources.*;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
+import org.apache.hadoop.ipc.ExternalCall;
|
|
|
import org.apache.hadoop.ipc.RetriableException;
|
|
|
-import org.apache.hadoop.ipc.Server;
|
|
|
import org.apache.hadoop.net.Node;
|
|
|
import org.apache.hadoop.net.NodeBase;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
@@ -101,39 +105,39 @@ public class NamenodeWebHdfsMethods {
|
|
|
public static final Log LOG = LogFactory.getLog(NamenodeWebHdfsMethods.class);
|
|
|
|
|
|
private static final UriFsPathParam ROOT = new UriFsPathParam("");
|
|
|
-
|
|
|
- private static final ThreadLocal<String> REMOTE_ADDRESS = new ThreadLocal<String>();
|
|
|
-
|
|
|
- /** @return the remote client address. */
|
|
|
- public static String getRemoteAddress() {
|
|
|
- return REMOTE_ADDRESS.get();
|
|
|
- }
|
|
|
-
|
|
|
- public static InetAddress getRemoteIp() {
|
|
|
- try {
|
|
|
- return InetAddress.getByName(getRemoteAddress());
|
|
|
- } catch (Exception e) {
|
|
|
- return null;
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
- /**
|
|
|
- * Returns true if a WebHdfs request is in progress. Akin to
|
|
|
- * {@link Server#isRpcInvocation()}.
|
|
|
- */
|
|
|
- public static boolean isWebHdfsInvocation() {
|
|
|
- return getRemoteAddress() != null;
|
|
|
- }
|
|
|
+ private volatile Boolean useIpcCallq;
|
|
|
+ private String scheme;
|
|
|
+ private Principal userPrincipal;
|
|
|
+ private String remoteAddr;
|
|
|
|
|
|
private @Context ServletContext context;
|
|
|
- private @Context HttpServletRequest request;
|
|
|
private @Context HttpServletResponse response;
|
|
|
|
|
|
+ public NamenodeWebHdfsMethods(@Context HttpServletRequest request) {
|
|
|
+ // the request object is a proxy to thread-locals so we have to extract
|
|
|
+ // what we want from it since the external call will be processed in a
|
|
|
+ // different thread.
|
|
|
+ scheme = request.getScheme();
|
|
|
+ userPrincipal = request.getUserPrincipal();
|
|
|
+ // get the remote address, if coming in via a trusted proxy server then
|
|
|
+ // the address with be that of the proxied client
|
|
|
+ remoteAddr = JspHelper.getRemoteAddr(request);
|
|
|
+ }
|
|
|
+
|
|
|
private void init(final UserGroupInformation ugi,
|
|
|
final DelegationParam delegation,
|
|
|
final UserParam username, final DoAsParam doAsUser,
|
|
|
final UriFsPathParam path, final HttpOpParam<?> op,
|
|
|
final Param<?, ?>... parameters) {
|
|
|
+ if (useIpcCallq == null) {
|
|
|
+ Configuration conf =
|
|
|
+ (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
|
|
|
+ useIpcCallq = conf.getBoolean(
|
|
|
+ DFSConfigKeys.DFS_WEBHDFS_USE_IPC_CALLQ,
|
|
|
+ DFSConfigKeys.DFS_WEBHDFS_USE_IPC_CALLQ_DEFAULT);
|
|
|
+ }
|
|
|
+
|
|
|
if (LOG.isTraceEnabled()) {
|
|
|
LOG.trace("HTTP " + op.getValue().getType() + ": " + op + ", " + path
|
|
|
+ ", ugi=" + ugi + ", " + username + ", " + doAsUser
|
|
@@ -142,16 +146,8 @@ public class NamenodeWebHdfsMethods {
|
|
|
|
|
|
//clear content type
|
|
|
response.setContentType(null);
|
|
|
-
|
|
|
- // set the remote address, if coming in via a trust proxy server then
|
|
|
- // the address with be that of the proxied client
|
|
|
- REMOTE_ADDRESS.set(JspHelper.getRemoteAddr(request));
|
|
|
}
|
|
|
|
|
|
- private void reset() {
|
|
|
- REMOTE_ADDRESS.set(null);
|
|
|
- }
|
|
|
-
|
|
|
private static NamenodeProtocols getRPCServer(NameNode namenode)
|
|
|
throws IOException {
|
|
|
final NamenodeProtocols np = namenode.getRpcServer();
|
|
@@ -160,11 +156,63 @@ public class NamenodeWebHdfsMethods {
|
|
|
}
|
|
|
return np;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ private <T> T doAs(final UserGroupInformation ugi,
|
|
|
+ final PrivilegedExceptionAction<T> action)
|
|
|
+ throws IOException, InterruptedException {
|
|
|
+ return useIpcCallq ? doAsExternalCall(ugi, action) : ugi.doAs(action);
|
|
|
+ }
|
|
|
+
|
|
|
+ private <T> T doAsExternalCall(final UserGroupInformation ugi,
|
|
|
+ final PrivilegedExceptionAction<T> action)
|
|
|
+ throws IOException, InterruptedException {
|
|
|
+ // set the remote address, if coming in via a trust proxy server then
|
|
|
+ // the address with be that of the proxied client
|
|
|
+ ExternalCall<T> call = new ExternalCall<T>(action){
|
|
|
+ @Override
|
|
|
+ public UserGroupInformation getRemoteUser() {
|
|
|
+ return ugi;
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public String getProtocol() {
|
|
|
+ return "webhdfs";
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public String getHostAddress() {
|
|
|
+ return remoteAddr;
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public InetAddress getHostInetAddress() {
|
|
|
+ try {
|
|
|
+ return InetAddress.getByName(getHostAddress());
|
|
|
+ } catch (UnknownHostException e) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ };
|
|
|
+ final NameNode namenode = (NameNode)context.getAttribute("name.node");
|
|
|
+ namenode.queueExternalCall(call);
|
|
|
+ T result = null;
|
|
|
+ try {
|
|
|
+ result = call.get();
|
|
|
+ } catch (ExecutionException ee) {
|
|
|
+ Throwable t = ee.getCause();
|
|
|
+ if (t instanceof RuntimeException) {
|
|
|
+ throw (RuntimeException)t;
|
|
|
+ } else if (t instanceof IOException) {
|
|
|
+ throw (IOException)t;
|
|
|
+ } else {
|
|
|
+ throw new IOException(t);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return result;
|
|
|
+ }
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
static DatanodeInfo chooseDatanode(final NameNode namenode,
|
|
|
final String path, final HttpOpParam.Op op, final long openOffset,
|
|
|
- final long blocksize, final String excludeDatanodes) throws IOException {
|
|
|
+ final long blocksize, final String excludeDatanodes,
|
|
|
+ final String remoteAddr) throws IOException {
|
|
|
FSNamesystem fsn = namenode.getNamesystem();
|
|
|
if (fsn == null) {
|
|
|
throw new IOException("Namesystem has not been intialized yet.");
|
|
@@ -188,7 +236,7 @@ public class NamenodeWebHdfsMethods {
|
|
|
if (op == PutOpParam.Op.CREATE) {
|
|
|
//choose a datanode near to client
|
|
|
final DatanodeDescriptor clientNode = bm.getDatanodeManager(
|
|
|
- ).getDatanodeByHost(getRemoteAddress());
|
|
|
+ ).getDatanodeByHost(remoteAddr);
|
|
|
if (clientNode != null) {
|
|
|
final DatanodeStorageInfo[] storages = bm.chooseTarget4WebHDFS(
|
|
|
path, clientNode, excludes, blocksize);
|
|
@@ -251,7 +299,8 @@ public class NamenodeWebHdfsMethods {
|
|
|
return null;
|
|
|
}
|
|
|
final Token<? extends TokenIdentifier> t = c.getAllTokens().iterator().next();
|
|
|
- Text kind = request.getScheme().equals("http") ? WebHdfsConstants.WEBHDFS_TOKEN_KIND
|
|
|
+ Text kind = scheme.equals("http")
|
|
|
+ ? WebHdfsConstants.WEBHDFS_TOKEN_KIND
|
|
|
: WebHdfsConstants.SWEBHDFS_TOKEN_KIND;
|
|
|
t.setKind(kind);
|
|
|
return t;
|
|
@@ -265,7 +314,7 @@ public class NamenodeWebHdfsMethods {
|
|
|
final Param<?, ?>... parameters) throws URISyntaxException, IOException {
|
|
|
final DatanodeInfo dn;
|
|
|
dn = chooseDatanode(namenode, path, op, openOffset, blocksize,
|
|
|
- excludeDatanodes);
|
|
|
+ excludeDatanodes, remoteAddr);
|
|
|
if (dn == null) {
|
|
|
throw new IOException("Failed to find datanode, suggest to check cluster"
|
|
|
+ " health. excludeDatanodes=" + excludeDatanodes);
|
|
@@ -281,7 +330,7 @@ public class NamenodeWebHdfsMethods {
|
|
|
} else {
|
|
|
//generate a token
|
|
|
final Token<? extends TokenIdentifier> t = generateDelegationToken(
|
|
|
- namenode, ugi, request.getUserPrincipal().getName());
|
|
|
+ namenode, ugi, userPrincipal.getName());
|
|
|
delegationQuery = "&" + new DelegationParam(t.encodeToUrlString());
|
|
|
}
|
|
|
final String query = op.toQueryString() + delegationQuery
|
|
@@ -289,7 +338,6 @@ public class NamenodeWebHdfsMethods {
|
|
|
+ Param.toSortedString("&", parameters);
|
|
|
final String uripath = WebHdfsFileSystem.PATH_PREFIX + path;
|
|
|
|
|
|
- final String scheme = request.getScheme();
|
|
|
int port = "http".equals(scheme) ? dn.getInfoPort() : dn
|
|
|
.getInfoSecurePort();
|
|
|
final URI uri = new URI(scheme, null, dn.getHostName(), port, uripath,
|
|
@@ -432,10 +480,9 @@ public class NamenodeWebHdfsMethods {
|
|
|
aclPermission, xattrName, xattrValue, xattrSetFlag, snapshotName,
|
|
|
oldSnapshotName, excludeDatanodes, createFlagParam);
|
|
|
|
|
|
- return ugi.doAs(new PrivilegedExceptionAction<Response>() {
|
|
|
+ return doAs(ugi, new PrivilegedExceptionAction<Response>() {
|
|
|
@Override
|
|
|
public Response run() throws IOException, URISyntaxException {
|
|
|
- try {
|
|
|
return put(ugi, delegation, username, doAsUser,
|
|
|
path.getAbsolutePath(), op, destination, owner, group,
|
|
|
permission, overwrite, bufferSize, replication, blockSize,
|
|
@@ -443,9 +490,6 @@ public class NamenodeWebHdfsMethods {
|
|
|
delegationTokenArgument, aclPermission, xattrName, xattrValue,
|
|
|
xattrSetFlag, snapshotName, oldSnapshotName, excludeDatanodes,
|
|
|
createFlagParam);
|
|
|
- } finally {
|
|
|
- reset();
|
|
|
- }
|
|
|
}
|
|
|
});
|
|
|
}
|
|
@@ -662,16 +706,12 @@ public class NamenodeWebHdfsMethods {
|
|
|
init(ugi, delegation, username, doAsUser, path, op, concatSrcs, bufferSize,
|
|
|
excludeDatanodes, newLength);
|
|
|
|
|
|
- return ugi.doAs(new PrivilegedExceptionAction<Response>() {
|
|
|
+ return doAs(ugi, new PrivilegedExceptionAction<Response>() {
|
|
|
@Override
|
|
|
public Response run() throws IOException, URISyntaxException {
|
|
|
- try {
|
|
|
return post(ugi, delegation, username, doAsUser,
|
|
|
path.getAbsolutePath(), op, concatSrcs, bufferSize,
|
|
|
excludeDatanodes, newLength);
|
|
|
- } finally {
|
|
|
- reset();
|
|
|
- }
|
|
|
}
|
|
|
});
|
|
|
}
|
|
@@ -802,17 +842,13 @@ public class NamenodeWebHdfsMethods {
|
|
|
renewer, bufferSize, xattrEncoding, excludeDatanodes, fsAction,
|
|
|
tokenKind, tokenService);
|
|
|
|
|
|
- return ugi.doAs(new PrivilegedExceptionAction<Response>() {
|
|
|
+ return doAs(ugi, new PrivilegedExceptionAction<Response>() {
|
|
|
@Override
|
|
|
public Response run() throws IOException, URISyntaxException {
|
|
|
- try {
|
|
|
return get(ugi, delegation, username, doAsUser,
|
|
|
path.getAbsolutePath(), op, offset, length, renewer, bufferSize,
|
|
|
xattrNames, xattrEncoding, excludeDatanodes, fsAction, tokenKind,
|
|
|
tokenService);
|
|
|
- } finally {
|
|
|
- reset();
|
|
|
- }
|
|
|
}
|
|
|
});
|
|
|
}
|
|
@@ -1058,15 +1094,11 @@ public class NamenodeWebHdfsMethods {
|
|
|
|
|
|
init(ugi, delegation, username, doAsUser, path, op, recursive, snapshotName);
|
|
|
|
|
|
- return ugi.doAs(new PrivilegedExceptionAction<Response>() {
|
|
|
+ return doAs(ugi, new PrivilegedExceptionAction<Response>() {
|
|
|
@Override
|
|
|
public Response run() throws IOException {
|
|
|
- try {
|
|
|
return delete(ugi, delegation, username, doAsUser,
|
|
|
path.getAbsolutePath(), op, recursive, snapshotName);
|
|
|
- } finally {
|
|
|
- reset();
|
|
|
- }
|
|
|
}
|
|
|
});
|
|
|
}
|