|
@@ -69,6 +69,7 @@ import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.DelegationParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.DeleteOpParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.DestinationParam;
|
|
|
+import org.apache.hadoop.hdfs.web.resources.DoAsParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.GetOpParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.GroupParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
|
|
@@ -116,6 +117,21 @@ public class NamenodeWebHdfsMethods {
|
|
|
private @Context HttpServletRequest request;
|
|
|
private @Context HttpServletResponse response;
|
|
|
|
|
|
+ private void init(final UserGroupInformation ugi,
|
|
|
+ final DelegationParam delegation,
|
|
|
+ final UserParam username, final DoAsParam doAsUser,
|
|
|
+ final UriFsPathParam path, final HttpOpParam<?> op,
|
|
|
+ final Param<?, ?>... parameters) throws IOException {
|
|
|
+ if (LOG.isTraceEnabled()) {
|
|
|
+ LOG.trace("HTTP " + op.getValue().getType() + ": " + op + ", " + path
|
|
|
+ + ", ugi=" + ugi + ", " + username + ", " + doAsUser
|
|
|
+ + Param.toSortedString(", ", parameters));
|
|
|
+ }
|
|
|
+
|
|
|
+ //clear content type
|
|
|
+ response.setContentType(null);
|
|
|
+ }
|
|
|
+
|
|
|
private static DatanodeInfo chooseDatanode(final NameNode namenode,
|
|
|
final String path, final HttpOpParam.Op op, final long openOffset,
|
|
|
Configuration conf) throws IOException {
|
|
@@ -161,6 +177,7 @@ public class NamenodeWebHdfsMethods {
|
|
|
|
|
|
private URI redirectURI(final NameNode namenode,
|
|
|
final UserGroupInformation ugi, final DelegationParam delegation,
|
|
|
+ final UserParam username, final DoAsParam doAsUser,
|
|
|
final String path, final HttpOpParam.Op op, final long openOffset,
|
|
|
final Param<?, ?>... parameters) throws URISyntaxException, IOException {
|
|
|
final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
|
|
@@ -169,7 +186,7 @@ public class NamenodeWebHdfsMethods {
|
|
|
final String delegationQuery;
|
|
|
if (!UserGroupInformation.isSecurityEnabled()) {
|
|
|
//security disabled
|
|
|
- delegationQuery = "";
|
|
|
+ delegationQuery = Param.toSortedString("&", doAsUser, username);
|
|
|
} else if (delegation.getValue() != null) {
|
|
|
//client has provided a token
|
|
|
delegationQuery = "&" + delegation;
|
|
@@ -179,8 +196,7 @@ public class NamenodeWebHdfsMethods {
|
|
|
namenode, ugi, request.getUserPrincipal().getName());
|
|
|
delegationQuery = "&" + new DelegationParam(t.encodeToUrlString());
|
|
|
}
|
|
|
- final String query = op.toQueryString()
|
|
|
- + '&' + new UserParam(ugi) + delegationQuery
|
|
|
+ final String query = op.toQueryString() + delegationQuery
|
|
|
+ Param.toSortedString("&", parameters);
|
|
|
final String uripath = WebHdfsFileSystem.PATH_PREFIX + path;
|
|
|
|
|
@@ -201,6 +217,10 @@ public class NamenodeWebHdfsMethods {
|
|
|
@Context final UserGroupInformation ugi,
|
|
|
@QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
|
|
|
final DelegationParam delegation,
|
|
|
+ @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
|
|
|
+ final UserParam username,
|
|
|
+ @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
|
|
|
+ final DoAsParam doAsUser,
|
|
|
@QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT)
|
|
|
final PutOpParam op,
|
|
|
@QueryParam(DestinationParam.NAME) @DefaultValue(DestinationParam.DEFAULT)
|
|
@@ -225,12 +245,13 @@ public class NamenodeWebHdfsMethods {
|
|
|
final AccessTimeParam accessTime,
|
|
|
@QueryParam(RenameOptionSetParam.NAME) @DefaultValue(RenameOptionSetParam.DEFAULT)
|
|
|
final RenameOptionSetParam renameOptions,
|
|
|
- @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT)
|
|
|
+ @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT)
|
|
|
final TokenArgumentParam delegationTokenArgument
|
|
|
) throws IOException, InterruptedException {
|
|
|
- return put(ugi, delegation, ROOT, op, destination, owner, group,
|
|
|
- permission, overwrite, bufferSize, replication, blockSize,
|
|
|
- modificationTime, accessTime, renameOptions, delegationTokenArgument);
|
|
|
+ return put(ugi, delegation, username, doAsUser, ROOT, op, destination,
|
|
|
+ owner, group, permission, overwrite, bufferSize, replication,
|
|
|
+ blockSize, modificationTime, accessTime, renameOptions,
|
|
|
+ delegationTokenArgument);
|
|
|
}
|
|
|
|
|
|
/** Handle HTTP PUT request. */
|
|
@@ -242,6 +263,10 @@ public class NamenodeWebHdfsMethods {
|
|
|
@Context final UserGroupInformation ugi,
|
|
|
@QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
|
|
|
final DelegationParam delegation,
|
|
|
+ @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
|
|
|
+ final UserParam username,
|
|
|
+ @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
|
|
|
+ final DoAsParam doAsUser,
|
|
|
@PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
|
|
|
@QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT)
|
|
|
final PutOpParam op,
|
|
@@ -267,19 +292,13 @@ public class NamenodeWebHdfsMethods {
|
|
|
final AccessTimeParam accessTime,
|
|
|
@QueryParam(RenameOptionSetParam.NAME) @DefaultValue(RenameOptionSetParam.DEFAULT)
|
|
|
final RenameOptionSetParam renameOptions,
|
|
|
- @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT)
|
|
|
+ @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT)
|
|
|
final TokenArgumentParam delegationTokenArgument
|
|
|
) throws IOException, InterruptedException {
|
|
|
|
|
|
- if (LOG.isTraceEnabled()) {
|
|
|
- LOG.trace(op + ": " + path + ", ugi=" + ugi
|
|
|
- + Param.toSortedString(", ", destination, owner, group, permission,
|
|
|
- overwrite, bufferSize, replication, blockSize,
|
|
|
- modificationTime, accessTime, renameOptions));
|
|
|
- }
|
|
|
-
|
|
|
- //clear content type
|
|
|
- response.setContentType(null);
|
|
|
+ init(ugi, delegation, username, doAsUser, path, op, destination, owner,
|
|
|
+ group, permission, overwrite, bufferSize, replication, blockSize,
|
|
|
+ modificationTime, accessTime, renameOptions, delegationTokenArgument);
|
|
|
|
|
|
return ugi.doAs(new PrivilegedExceptionAction<Response>() {
|
|
|
@Override
|
|
@@ -295,8 +314,8 @@ public class NamenodeWebHdfsMethods {
|
|
|
switch(op.getValue()) {
|
|
|
case CREATE:
|
|
|
{
|
|
|
- final URI uri = redirectURI(namenode, ugi, delegation, fullpath,
|
|
|
- op.getValue(), -1L,
|
|
|
+ final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser,
|
|
|
+ fullpath, op.getValue(), -1L,
|
|
|
permission, overwrite, bufferSize, replication, blockSize);
|
|
|
return Response.temporaryRedirect(uri).build();
|
|
|
}
|
|
@@ -379,12 +398,16 @@ public class NamenodeWebHdfsMethods {
|
|
|
@Context final UserGroupInformation ugi,
|
|
|
@QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
|
|
|
final DelegationParam delegation,
|
|
|
+ @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
|
|
|
+ final UserParam username,
|
|
|
+ @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
|
|
|
+ final DoAsParam doAsUser,
|
|
|
@QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT)
|
|
|
final PostOpParam op,
|
|
|
@QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
|
|
|
final BufferSizeParam bufferSize
|
|
|
) throws IOException, InterruptedException {
|
|
|
- return post(ugi, delegation, ROOT, op, bufferSize);
|
|
|
+ return post(ugi, delegation, username, doAsUser, ROOT, op, bufferSize);
|
|
|
}
|
|
|
|
|
|
/** Handle HTTP POST request. */
|
|
@@ -396,6 +419,10 @@ public class NamenodeWebHdfsMethods {
|
|
|
@Context final UserGroupInformation ugi,
|
|
|
@QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
|
|
|
final DelegationParam delegation,
|
|
|
+ @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
|
|
|
+ final UserParam username,
|
|
|
+ @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
|
|
|
+ final DoAsParam doAsUser,
|
|
|
@PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
|
|
|
@QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT)
|
|
|
final PostOpParam op,
|
|
@@ -403,13 +430,7 @@ public class NamenodeWebHdfsMethods {
|
|
|
final BufferSizeParam bufferSize
|
|
|
) throws IOException, InterruptedException {
|
|
|
|
|
|
- if (LOG.isTraceEnabled()) {
|
|
|
- LOG.trace(op + ": " + path + ", ugi=" + ugi
|
|
|
- + Param.toSortedString(", ", bufferSize));
|
|
|
- }
|
|
|
-
|
|
|
- //clear content type
|
|
|
- response.setContentType(null);
|
|
|
+ init(ugi, delegation, username, doAsUser, path, op, bufferSize);
|
|
|
|
|
|
return ugi.doAs(new PrivilegedExceptionAction<Response>() {
|
|
|
@Override
|
|
@@ -423,8 +444,8 @@ public class NamenodeWebHdfsMethods {
|
|
|
switch(op.getValue()) {
|
|
|
case APPEND:
|
|
|
{
|
|
|
- final URI uri = redirectURI(namenode, ugi, delegation, fullpath,
|
|
|
- op.getValue(), -1L, bufferSize);
|
|
|
+ final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser,
|
|
|
+ fullpath, op.getValue(), -1L, bufferSize);
|
|
|
return Response.temporaryRedirect(uri).build();
|
|
|
}
|
|
|
default:
|
|
@@ -446,6 +467,10 @@ public class NamenodeWebHdfsMethods {
|
|
|
@Context final UserGroupInformation ugi,
|
|
|
@QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
|
|
|
final DelegationParam delegation,
|
|
|
+ @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
|
|
|
+ final UserParam username,
|
|
|
+ @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
|
|
|
+ final DoAsParam doAsUser,
|
|
|
@QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT)
|
|
|
final GetOpParam op,
|
|
|
@QueryParam(OffsetParam.NAME) @DefaultValue(OffsetParam.DEFAULT)
|
|
@@ -457,7 +482,8 @@ public class NamenodeWebHdfsMethods {
|
|
|
@QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
|
|
|
final BufferSizeParam bufferSize
|
|
|
) throws IOException, URISyntaxException, InterruptedException {
|
|
|
- return get(ugi, delegation, ROOT, op, offset, length, renewer, bufferSize);
|
|
|
+ return get(ugi, delegation, username, doAsUser, ROOT, op,
|
|
|
+ offset, length, renewer, bufferSize);
|
|
|
}
|
|
|
|
|
|
/** Handle HTTP GET request. */
|
|
@@ -468,6 +494,10 @@ public class NamenodeWebHdfsMethods {
|
|
|
@Context final UserGroupInformation ugi,
|
|
|
@QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
|
|
|
final DelegationParam delegation,
|
|
|
+ @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
|
|
|
+ final UserParam username,
|
|
|
+ @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
|
|
|
+ final DoAsParam doAsUser,
|
|
|
@PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
|
|
|
@QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT)
|
|
|
final GetOpParam op,
|
|
@@ -481,13 +511,8 @@ public class NamenodeWebHdfsMethods {
|
|
|
final BufferSizeParam bufferSize
|
|
|
) throws IOException, InterruptedException {
|
|
|
|
|
|
- if (LOG.isTraceEnabled()) {
|
|
|
- LOG.trace(op + ": " + path + ", ugi=" + ugi
|
|
|
- + Param.toSortedString(", ", offset, length, renewer, bufferSize));
|
|
|
- }
|
|
|
-
|
|
|
- //clear content type
|
|
|
- response.setContentType(null);
|
|
|
+ init(ugi, delegation, username, doAsUser, path, op,
|
|
|
+ offset, length, renewer, bufferSize);
|
|
|
|
|
|
return ugi.doAs(new PrivilegedExceptionAction<Response>() {
|
|
|
@Override
|
|
@@ -502,8 +527,8 @@ public class NamenodeWebHdfsMethods {
|
|
|
switch(op.getValue()) {
|
|
|
case OPEN:
|
|
|
{
|
|
|
- final URI uri = redirectURI(namenode, ugi, delegation, fullpath,
|
|
|
- op.getValue(), offset.getValue(), offset, length, bufferSize);
|
|
|
+ final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser,
|
|
|
+ fullpath, op.getValue(), offset.getValue(), offset, length, bufferSize);
|
|
|
return Response.temporaryRedirect(uri).build();
|
|
|
}
|
|
|
case GET_BLOCK_LOCATIONS:
|
|
@@ -538,17 +563,28 @@ public class NamenodeWebHdfsMethods {
|
|
|
}
|
|
|
case GETFILECHECKSUM:
|
|
|
{
|
|
|
- final URI uri = redirectURI(namenode, ugi, delegation, fullpath,
|
|
|
- op.getValue(), -1L);
|
|
|
+ final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser,
|
|
|
+ fullpath, op.getValue(), -1L);
|
|
|
return Response.temporaryRedirect(uri).build();
|
|
|
}
|
|
|
case GETDELEGATIONTOKEN:
|
|
|
{
|
|
|
+ if (delegation.getValue() != null) {
|
|
|
+ throw new IllegalArgumentException(delegation.getName()
|
|
|
+ + " parameter is not null.");
|
|
|
+ }
|
|
|
final Token<? extends TokenIdentifier> token = generateDelegationToken(
|
|
|
namenode, ugi, renewer.getValue());
|
|
|
final String js = JsonUtil.toJsonString(token);
|
|
|
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
|
|
|
}
|
|
|
+ case GETHOMEDIRECTORY:
|
|
|
+ {
|
|
|
+ final String js = JsonUtil.toJsonString(
|
|
|
+ org.apache.hadoop.fs.Path.class.getSimpleName(),
|
|
|
+ WebHdfsFileSystem.getHomeDirectoryString(ugi));
|
|
|
+ return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
|
|
|
+ }
|
|
|
default:
|
|
|
throw new UnsupportedOperationException(op + " is not supported");
|
|
|
}
|
|
@@ -610,12 +646,18 @@ public class NamenodeWebHdfsMethods {
|
|
|
@Produces(MediaType.APPLICATION_JSON)
|
|
|
public Response deleteRoot(
|
|
|
@Context final UserGroupInformation ugi,
|
|
|
+ @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
|
|
|
+ final DelegationParam delegation,
|
|
|
+ @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
|
|
|
+ final UserParam username,
|
|
|
+ @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
|
|
|
+ final DoAsParam doAsUser,
|
|
|
@QueryParam(DeleteOpParam.NAME) @DefaultValue(DeleteOpParam.DEFAULT)
|
|
|
final DeleteOpParam op,
|
|
|
@QueryParam(RecursiveParam.NAME) @DefaultValue(RecursiveParam.DEFAULT)
|
|
|
final RecursiveParam recursive
|
|
|
) throws IOException, InterruptedException {
|
|
|
- return delete(ugi, ROOT, op, recursive);
|
|
|
+ return delete(ugi, delegation, username, doAsUser, ROOT, op, recursive);
|
|
|
}
|
|
|
|
|
|
/** Handle HTTP DELETE request. */
|
|
@@ -624,6 +666,12 @@ public class NamenodeWebHdfsMethods {
|
|
|
@Produces(MediaType.APPLICATION_JSON)
|
|
|
public Response delete(
|
|
|
@Context final UserGroupInformation ugi,
|
|
|
+ @QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
|
|
|
+ final DelegationParam delegation,
|
|
|
+ @QueryParam(UserParam.NAME) @DefaultValue(UserParam.DEFAULT)
|
|
|
+ final UserParam username,
|
|
|
+ @QueryParam(DoAsParam.NAME) @DefaultValue(DoAsParam.DEFAULT)
|
|
|
+ final DoAsParam doAsUser,
|
|
|
@PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
|
|
|
@QueryParam(DeleteOpParam.NAME) @DefaultValue(DeleteOpParam.DEFAULT)
|
|
|
final DeleteOpParam op,
|
|
@@ -631,13 +679,7 @@ public class NamenodeWebHdfsMethods {
|
|
|
final RecursiveParam recursive
|
|
|
) throws IOException, InterruptedException {
|
|
|
|
|
|
- if (LOG.isTraceEnabled()) {
|
|
|
- LOG.trace(op + ": " + path + ", ugi=" + ugi
|
|
|
- + Param.toSortedString(", ", recursive));
|
|
|
- }
|
|
|
-
|
|
|
- //clear content type
|
|
|
- response.setContentType(null);
|
|
|
+ init(ugi, delegation, username, doAsUser, path, op, recursive);
|
|
|
|
|
|
return ugi.doAs(new PrivilegedExceptionAction<Response>() {
|
|
|
@Override
|