|
@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.federation.router;
|
|
|
|
|
|
import static org.apache.hadoop.util.StringUtils.getTrimmedStringCollection;
|
|
|
|
|
|
-import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
@@ -27,13 +26,10 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
|
|
import org.apache.hadoop.hdfs.server.common.JspHelper;
|
|
|
-import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
|
|
|
-import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext;
|
|
|
-import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
|
|
|
+import org.apache.hadoop.hdfs.server.federation.router.security.RouterSecurityManager;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
|
|
|
|
|
|
import javax.servlet.http.HttpServletRequest;
|
|
|
-import javax.servlet.http.HttpServletResponse;
|
|
|
import javax.ws.rs.Path;
|
|
|
import javax.ws.rs.core.Context;
|
|
|
import javax.ws.rs.core.MediaType;
|
|
@@ -42,7 +38,6 @@ import javax.ws.rs.core.Response;
|
|
|
import com.sun.jersey.spi.container.ResourceFilters;
|
|
|
import org.apache.hadoop.hdfs.web.JsonUtil;
|
|
|
import org.apache.hadoop.hdfs.web.ParamFilter;
|
|
|
-import org.apache.hadoop.hdfs.web.URLConnectionFactory;
|
|
|
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
|
|
|
import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.AclPermissionParam;
|
|
@@ -91,6 +86,7 @@ import org.apache.hadoop.hdfs.web.resources.XAttrValueParam;
|
|
|
import org.apache.hadoop.ipc.ExternalCall;
|
|
|
import org.apache.hadoop.ipc.RetriableException;
|
|
|
import org.apache.hadoop.net.Node;
|
|
|
+import org.apache.hadoop.security.Credentials;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
@@ -99,12 +95,8 @@ import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
-import java.net.HttpURLConnection;
|
|
|
import java.net.URI;
|
|
|
import java.net.URISyntaxException;
|
|
|
-import java.net.URL;
|
|
|
-import java.net.URLDecoder;
|
|
|
-import java.security.PrivilegedAction;
|
|
|
import java.util.Collection;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.List;
|
|
@@ -224,7 +216,11 @@ public class RouterWebHdfsMethods extends NamenodeWebHdfsMethods {
|
|
|
case CREATE:
|
|
|
{
|
|
|
final Router router = getRouter();
|
|
|
- final URI uri = redirectURI(router, fullpath);
|
|
|
+ final URI uri = redirectURI(router, ugi, delegation, username,
|
|
|
+ doAsUser, fullpath, op.getValue(), -1L,
|
|
|
+ exclDatanodes.getValue(), permission, unmaskedPermission,
|
|
|
+ overwrite, bufferSize, replication, blockSize, createParent,
|
|
|
+ createFlagParam);
|
|
|
if (!noredirectParam.getValue()) {
|
|
|
return Response.temporaryRedirect(uri)
|
|
|
.type(MediaType.APPLICATION_OCTET_STREAM).build();
|
|
@@ -366,6 +362,7 @@ public class RouterWebHdfsMethods extends NamenodeWebHdfsMethods {
|
|
|
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
|
|
|
}
|
|
|
}
|
|
|
+ case GETDELEGATIONTOKEN:
|
|
|
case GET_BLOCK_LOCATIONS:
|
|
|
case GETFILESTATUS:
|
|
|
case LISTSTATUS:
|
|
@@ -389,104 +386,6 @@ public class RouterWebHdfsMethods extends NamenodeWebHdfsMethods {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Get the redirect URI from the Namenode responsible for a path.
|
|
|
- * @param router Router to check.
|
|
|
- * @param path Path to get location for.
|
|
|
- * @return URI returned by the Namenode.
|
|
|
- * @throws IOException If it cannot get the redirect URI.
|
|
|
- */
|
|
|
- private URI redirectURI(final Router router, final String path)
|
|
|
- throws IOException {
|
|
|
- // Forward the request to the proper Namenode
|
|
|
- final HttpURLConnection conn = forwardRequest(router, path);
|
|
|
- try {
|
|
|
- conn.setInstanceFollowRedirects(false);
|
|
|
- conn.setDoOutput(true);
|
|
|
- conn.connect();
|
|
|
-
|
|
|
- // Read the reply from the Namenode
|
|
|
- int responseCode = conn.getResponseCode();
|
|
|
- if (responseCode != HttpServletResponse.SC_TEMPORARY_REDIRECT) {
|
|
|
- LOG.info("We expected a redirection from the Namenode, not {}",
|
|
|
- responseCode);
|
|
|
- return null;
|
|
|
- }
|
|
|
-
|
|
|
- // Extract the redirect location and return it
|
|
|
- String redirectLocation = conn.getHeaderField("Location");
|
|
|
- try {
|
|
|
- // We modify the namenode location and the path
|
|
|
- redirectLocation = redirectLocation
|
|
|
- .replaceAll("(?<=[?&;])namenoderpcaddress=.*?(?=[&;])",
|
|
|
- "namenoderpcaddress=" + router.getRouterId())
|
|
|
- .replaceAll("(?<=[/])webhdfs/v1/.*?(?=[?])",
|
|
|
- "webhdfs/v1" + path);
|
|
|
- return new URI(redirectLocation);
|
|
|
- } catch (URISyntaxException e) {
|
|
|
- LOG.error("Cannot parse redirect location {}", redirectLocation);
|
|
|
- }
|
|
|
- } finally {
|
|
|
- if (conn != null) {
|
|
|
- conn.disconnect();
|
|
|
- }
|
|
|
- }
|
|
|
- return null;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Forwards a request to a subcluster.
|
|
|
- * @param router Router to check.
|
|
|
- * @param path Path in HDFS.
|
|
|
- * @return Reply from the subcluster.
|
|
|
- * @throws IOException
|
|
|
- */
|
|
|
- private HttpURLConnection forwardRequest(
|
|
|
- final Router router, final String path) throws IOException {
|
|
|
- final Configuration conf =
|
|
|
- (Configuration)getContext().getAttribute(JspHelper.CURRENT_CONF);
|
|
|
- URLConnectionFactory connectionFactory =
|
|
|
- URLConnectionFactory.newDefaultURLConnectionFactory(conf);
|
|
|
-
|
|
|
- // Find the namespace responsible for a path
|
|
|
- final RouterRpcServer rpcServer = getRPCServer(router);
|
|
|
- RemoteLocation createLoc = rpcServer.getCreateLocation(path);
|
|
|
- String nsId = createLoc.getNameserviceId();
|
|
|
- String dest = createLoc.getDest();
|
|
|
- ActiveNamenodeResolver nnResolver = router.getNamenodeResolver();
|
|
|
- List<? extends FederationNamenodeContext> namenodes =
|
|
|
- nnResolver.getNamenodesForNameserviceId(nsId);
|
|
|
-
|
|
|
- // Go over the namenodes responsible for that namespace
|
|
|
- for (FederationNamenodeContext namenode : namenodes) {
|
|
|
- try {
|
|
|
- // Generate the request for the namenode
|
|
|
- String nnWebAddress = namenode.getWebAddress();
|
|
|
- String[] nnWebAddressSplit = nnWebAddress.split(":");
|
|
|
- String host = nnWebAddressSplit[0];
|
|
|
- int port = Integer.parseInt(nnWebAddressSplit[1]);
|
|
|
-
|
|
|
- // Avoid double-encoding here
|
|
|
- query = URLDecoder.decode(query, "UTF-8");
|
|
|
- URI uri = new URI(getScheme(), null, host, port,
|
|
|
- reqPath + dest, query, null);
|
|
|
- URL url = uri.toURL();
|
|
|
-
|
|
|
- // Send a request to the proper Namenode
|
|
|
- final HttpURLConnection conn =
|
|
|
- (HttpURLConnection)connectionFactory.openConnection(url);
|
|
|
- conn.setRequestMethod(method);
|
|
|
-
|
|
|
- connectionFactory.destroy();
|
|
|
- return conn;
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.error("Cannot redirect request to {}", namenode, e);
|
|
|
- }
|
|
|
- }
|
|
|
- connectionFactory.destroy();
|
|
|
- return null;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Get a URI to redirect an operation to.
|
|
|
* @param router Router to check.
|
|
@@ -526,7 +425,7 @@ public class RouterWebHdfsMethods extends NamenodeWebHdfsMethods {
|
|
|
} else {
|
|
|
// generate a token
|
|
|
final Token<? extends TokenIdentifier> t = generateDelegationToken(
|
|
|
- router, ugi, request.getUserPrincipal().getName());
|
|
|
+ ugi, ugi.getUserName());
|
|
|
delegationQuery = "&delegation=" + t.encodeToUrlString();
|
|
|
}
|
|
|
|
|
@@ -552,19 +451,17 @@ public class RouterWebHdfsMethods extends NamenodeWebHdfsMethods {
|
|
|
// We need to get the DNs as a privileged user
|
|
|
final RouterRpcServer rpcServer = getRPCServer(router);
|
|
|
UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
|
|
|
+ RouterRpcServer.setCurrentUser(loginUser);
|
|
|
|
|
|
- DatanodeInfo[] dns = loginUser.doAs(
|
|
|
- new PrivilegedAction<DatanodeInfo[]>() {
|
|
|
- @Override
|
|
|
- public DatanodeInfo[] run() {
|
|
|
- try {
|
|
|
- return rpcServer.getDatanodeReport(DatanodeReportType.LIVE);
|
|
|
- } catch (IOException e) {
|
|
|
- LOG.error("Cannot get the datanodes from the RPC server", e);
|
|
|
- return null;
|
|
|
- }
|
|
|
- }
|
|
|
- });
|
|
|
+ DatanodeInfo[] dns = null;
|
|
|
+ try {
|
|
|
+ dns = rpcServer.getDatanodeReport(DatanodeReportType.LIVE);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.error("Cannot get the datanodes from the RPC server", e);
|
|
|
+ } finally {
|
|
|
+ // Reset ugi to remote user for remaining operations.
|
|
|
+ RouterRpcServer.resetCurrentUser();
|
|
|
+ }
|
|
|
|
|
|
HashSet<Node> excludes = new HashSet<Node>();
|
|
|
if (excludeDatanodes != null) {
|
|
@@ -646,17 +543,19 @@ public class RouterWebHdfsMethods extends NamenodeWebHdfsMethods {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Generate the delegation tokens for this request.
|
|
|
- * @param router Router.
|
|
|
+ * Generate the credentials for this request.
|
|
|
* @param ugi User group information.
|
|
|
* @param renewer Who is asking for the renewal.
|
|
|
- * @return The delegation tokens.
|
|
|
- * @throws IOException If it cannot create the tokens.
|
|
|
+ * @return Credentials holding delegation token.
|
|
|
+ * @throws IOException If it cannot create the credentials.
|
|
|
*/
|
|
|
- private Token<? extends TokenIdentifier> generateDelegationToken(
|
|
|
- final Router router, final UserGroupInformation ugi,
|
|
|
+ @Override
|
|
|
+ public Credentials createCredentials(
|
|
|
+ final UserGroupInformation ugi,
|
|
|
final String renewer) throws IOException {
|
|
|
- throw new UnsupportedOperationException("TODO Generate token for ugi=" +
|
|
|
- ugi + " request=" + request);
|
|
|
+ final Router router = (Router)getContext().getAttribute("name.node");
|
|
|
+ final Credentials c = RouterSecurityManager.createCredentials(router, ugi,
|
|
|
+ renewer != null? renewer: ugi.getShortUserName());
|
|
|
+ return c;
|
|
|
}
|
|
|
}
|