|
@@ -19,13 +19,13 @@ package org.apache.hadoop.hdfs.server.datanode.web.resources;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.io.InputStream;
|
|
|
-import java.net.InetSocketAddress;
|
|
|
import java.net.URI;
|
|
|
import java.net.URISyntaxException;
|
|
|
import java.security.PrivilegedExceptionAction;
|
|
|
import java.util.EnumSet;
|
|
|
|
|
|
import javax.servlet.ServletContext;
|
|
|
+import javax.servlet.http.HttpServletRequest;
|
|
|
import javax.servlet.http.HttpServletResponse;
|
|
|
import javax.ws.rs.Consumes;
|
|
|
import javax.ws.rs.DefaultValue;
|
|
@@ -40,6 +40,7 @@ import javax.ws.rs.core.Context;
|
|
|
import javax.ws.rs.core.MediaType;
|
|
|
import javax.ws.rs.core.Response;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -48,12 +49,14 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.DFSClient;
|
|
|
+import org.apache.hadoop.hdfs.HAUtil;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
import org.apache.hadoop.hdfs.web.JsonUtil;
|
|
|
import org.apache.hadoop.hdfs.web.ParamFilter;
|
|
|
+import org.apache.hadoop.hdfs.web.SWebHdfsFileSystem;
|
|
|
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
|
|
|
import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
|
|
@@ -61,7 +64,7 @@ import org.apache.hadoop.hdfs.web.resources.DelegationParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.GetOpParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.LengthParam;
|
|
|
-import org.apache.hadoop.hdfs.web.resources.NamenodeRpcAddressParam;
|
|
|
+import org.apache.hadoop.hdfs.web.resources.NamenodeAddressParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.OffsetParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.OverwriteParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.Param;
|
|
@@ -71,6 +74,7 @@ import org.apache.hadoop.hdfs.web.resources.PutOpParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.UriFsPathParam;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
+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;
|
|
@@ -86,18 +90,19 @@ public class DatanodeWebHdfsMethods {
|
|
|
private static final UriFsPathParam ROOT = new UriFsPathParam("");
|
|
|
|
|
|
private @Context ServletContext context;
|
|
|
+ private @Context HttpServletRequest request;
|
|
|
private @Context HttpServletResponse response;
|
|
|
|
|
|
private void init(final UserGroupInformation ugi,
|
|
|
- final DelegationParam delegation, final InetSocketAddress nnRpcAddr,
|
|
|
+ final DelegationParam delegation, final String nnId,
|
|
|
final UriFsPathParam path, final HttpOpParam<?> op,
|
|
|
final Param<?, ?>... parameters) throws IOException {
|
|
|
if (LOG.isTraceEnabled()) {
|
|
|
LOG.trace("HTTP " + op.getValue().getType() + ": " + op + ", " + path
|
|
|
+ ", ugi=" + ugi + Param.toSortedString(", ", parameters));
|
|
|
}
|
|
|
- if (nnRpcAddr == null) {
|
|
|
- throw new IllegalArgumentException(NamenodeRpcAddressParam.NAME
|
|
|
+ if (nnId == null) {
|
|
|
+ throw new IllegalArgumentException(NamenodeAddressParam.NAME
|
|
|
+ " is not specified.");
|
|
|
}
|
|
|
|
|
@@ -106,15 +111,32 @@ public class DatanodeWebHdfsMethods {
|
|
|
|
|
|
if (UserGroupInformation.isSecurityEnabled()) {
|
|
|
//add a token for RPC.
|
|
|
- final Token<DelegationTokenIdentifier> token =
|
|
|
- new Token<DelegationTokenIdentifier>();
|
|
|
- token.decodeFromUrlString(delegation.getValue());
|
|
|
- SecurityUtil.setTokenService(token, nnRpcAddr);
|
|
|
- token.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
|
|
|
+ final Token<DelegationTokenIdentifier> token = deserializeToken
|
|
|
+ (delegation.getValue(), nnId);
|
|
|
ugi.addToken(token);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ Token<DelegationTokenIdentifier> deserializeToken
|
|
|
+ (String delegation,String nnId) throws IOException {
|
|
|
+ final DataNode datanode = (DataNode) context.getAttribute("datanode");
|
|
|
+ final Configuration conf = datanode.getConf();
|
|
|
+ final Token<DelegationTokenIdentifier> token = new
|
|
|
+ Token<DelegationTokenIdentifier>();
|
|
|
+ token.decodeFromUrlString(delegation);
|
|
|
+ URI nnUri = URI.create(HdfsConstants.HDFS_URI_SCHEME +
|
|
|
+ "://" + nnId);
|
|
|
+ boolean isHA = HAUtil.isLogicalUri(conf, nnUri);
|
|
|
+ if (isHA) {
|
|
|
+ token.setService(HAUtil.buildTokenServiceForLogicalUri(nnUri));
|
|
|
+ } else {
|
|
|
+ token.setService(SecurityUtil.buildTokenService(nnUri));
|
|
|
+ }
|
|
|
+ token.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
|
|
|
+ return token;
|
|
|
+ }
|
|
|
+
|
|
|
/** Handle HTTP PUT request for the root. */
|
|
|
@PUT
|
|
|
@Path("/")
|
|
@@ -125,9 +147,9 @@ public class DatanodeWebHdfsMethods {
|
|
|
@Context final UserGroupInformation ugi,
|
|
|
@QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
|
|
|
final DelegationParam delegation,
|
|
|
- @QueryParam(NamenodeRpcAddressParam.NAME)
|
|
|
- @DefaultValue(NamenodeRpcAddressParam.DEFAULT)
|
|
|
- final NamenodeRpcAddressParam namenodeRpcAddress,
|
|
|
+ @QueryParam(NamenodeAddressParam.NAME)
|
|
|
+ @DefaultValue(NamenodeAddressParam.DEFAULT)
|
|
|
+ final NamenodeAddressParam namenode,
|
|
|
@QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT)
|
|
|
final PutOpParam op,
|
|
|
@QueryParam(PermissionParam.NAME) @DefaultValue(PermissionParam.DEFAULT)
|
|
@@ -141,8 +163,8 @@ public class DatanodeWebHdfsMethods {
|
|
|
@QueryParam(BlockSizeParam.NAME) @DefaultValue(BlockSizeParam.DEFAULT)
|
|
|
final BlockSizeParam blockSize
|
|
|
) throws IOException, InterruptedException {
|
|
|
- return put(in, ugi, delegation, namenodeRpcAddress, ROOT, op, permission,
|
|
|
- overwrite, bufferSize, replication, blockSize);
|
|
|
+ return put(in, ugi, delegation, namenode, ROOT, op, permission,
|
|
|
+ overwrite, bufferSize, replication, blockSize);
|
|
|
}
|
|
|
|
|
|
/** Handle HTTP PUT request. */
|
|
@@ -155,9 +177,9 @@ public class DatanodeWebHdfsMethods {
|
|
|
@Context final UserGroupInformation ugi,
|
|
|
@QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
|
|
|
final DelegationParam delegation,
|
|
|
- @QueryParam(NamenodeRpcAddressParam.NAME)
|
|
|
- @DefaultValue(NamenodeRpcAddressParam.DEFAULT)
|
|
|
- final NamenodeRpcAddressParam namenodeRpcAddress,
|
|
|
+ @QueryParam(NamenodeAddressParam.NAME)
|
|
|
+ @DefaultValue(NamenodeAddressParam.DEFAULT)
|
|
|
+ final NamenodeAddressParam namenode,
|
|
|
@PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
|
|
|
@QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT)
|
|
|
final PutOpParam op,
|
|
@@ -173,24 +195,22 @@ public class DatanodeWebHdfsMethods {
|
|
|
final BlockSizeParam blockSize
|
|
|
) throws IOException, InterruptedException {
|
|
|
|
|
|
- final InetSocketAddress nnRpcAddr = namenodeRpcAddress.getValue();
|
|
|
- init(ugi, delegation, nnRpcAddr, path, op, permission,
|
|
|
+ final String nnId = namenode.getValue();
|
|
|
+ init(ugi, delegation, nnId, path, op, permission,
|
|
|
overwrite, bufferSize, replication, blockSize);
|
|
|
|
|
|
return ugi.doAs(new PrivilegedExceptionAction<Response>() {
|
|
|
@Override
|
|
|
public Response run() throws IOException, URISyntaxException {
|
|
|
- return put(in, ugi, delegation, nnRpcAddr, path.getAbsolutePath(), op,
|
|
|
- permission, overwrite, bufferSize, replication, blockSize);
|
|
|
+ return put(in, nnId, path.getAbsolutePath(), op,
|
|
|
+ permission, overwrite, bufferSize, replication, blockSize);
|
|
|
}
|
|
|
});
|
|
|
}
|
|
|
|
|
|
private Response put(
|
|
|
final InputStream in,
|
|
|
- final UserGroupInformation ugi,
|
|
|
- final DelegationParam delegation,
|
|
|
- final InetSocketAddress nnRpcAddr,
|
|
|
+ final String nnId,
|
|
|
final String fullpath,
|
|
|
final PutOpParam op,
|
|
|
final PermissionParam permission,
|
|
@@ -208,7 +228,7 @@ public class DatanodeWebHdfsMethods {
|
|
|
conf.set(FsPermission.UMASK_LABEL, "000");
|
|
|
|
|
|
final int b = bufferSize.getValue(conf);
|
|
|
- DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
|
|
|
+ DFSClient dfsclient = newDfsClient(nnId, conf);
|
|
|
FSDataOutputStream out = null;
|
|
|
try {
|
|
|
out = new FSDataOutputStream(dfsclient.create(
|
|
@@ -225,9 +245,10 @@ public class DatanodeWebHdfsMethods {
|
|
|
IOUtils.cleanup(LOG, out);
|
|
|
IOUtils.cleanup(LOG, dfsclient);
|
|
|
}
|
|
|
- final InetSocketAddress nnHttpAddr = NameNode.getHttpAddress(conf);
|
|
|
- final URI uri = new URI(WebHdfsFileSystem.SCHEME, null,
|
|
|
- nnHttpAddr.getHostName(), nnHttpAddr.getPort(), fullpath, null, null);
|
|
|
+ final String scheme = "http".equals(request.getScheme()) ?
|
|
|
+ WebHdfsFileSystem.SCHEME : SWebHdfsFileSystem.SCHEME;
|
|
|
+ final URI uri = URI.create(String.format("%s://%s/%s", scheme,
|
|
|
+ nnId, fullpath));
|
|
|
return Response.created(uri).type(MediaType.APPLICATION_OCTET_STREAM).build();
|
|
|
}
|
|
|
default:
|
|
@@ -245,15 +266,15 @@ public class DatanodeWebHdfsMethods {
|
|
|
@Context final UserGroupInformation ugi,
|
|
|
@QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
|
|
|
final DelegationParam delegation,
|
|
|
- @QueryParam(NamenodeRpcAddressParam.NAME)
|
|
|
- @DefaultValue(NamenodeRpcAddressParam.DEFAULT)
|
|
|
- final NamenodeRpcAddressParam namenodeRpcAddress,
|
|
|
+ @QueryParam(NamenodeAddressParam.NAME)
|
|
|
+ @DefaultValue(NamenodeAddressParam.DEFAULT)
|
|
|
+ final NamenodeAddressParam namenode,
|
|
|
@QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT)
|
|
|
final PostOpParam op,
|
|
|
@QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
|
|
|
final BufferSizeParam bufferSize
|
|
|
) throws IOException, InterruptedException {
|
|
|
- return post(in, ugi, delegation, namenodeRpcAddress, ROOT, op, bufferSize);
|
|
|
+ return post(in, ugi, delegation, namenode, ROOT, op, bufferSize);
|
|
|
}
|
|
|
|
|
|
/** Handle HTTP POST request. */
|
|
@@ -266,9 +287,9 @@ public class DatanodeWebHdfsMethods {
|
|
|
@Context final UserGroupInformation ugi,
|
|
|
@QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
|
|
|
final DelegationParam delegation,
|
|
|
- @QueryParam(NamenodeRpcAddressParam.NAME)
|
|
|
- @DefaultValue(NamenodeRpcAddressParam.DEFAULT)
|
|
|
- final NamenodeRpcAddressParam namenodeRpcAddress,
|
|
|
+ @QueryParam(NamenodeAddressParam.NAME)
|
|
|
+ @DefaultValue(NamenodeAddressParam.DEFAULT)
|
|
|
+ final NamenodeAddressParam namenode,
|
|
|
@PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
|
|
|
@QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT)
|
|
|
final PostOpParam op,
|
|
@@ -276,23 +297,21 @@ public class DatanodeWebHdfsMethods {
|
|
|
final BufferSizeParam bufferSize
|
|
|
) throws IOException, InterruptedException {
|
|
|
|
|
|
- final InetSocketAddress nnRpcAddr = namenodeRpcAddress.getValue();
|
|
|
- init(ugi, delegation, nnRpcAddr, path, op, bufferSize);
|
|
|
+ final String nnId = namenode.getValue();
|
|
|
+ init(ugi, delegation, nnId, path, op, bufferSize);
|
|
|
|
|
|
return ugi.doAs(new PrivilegedExceptionAction<Response>() {
|
|
|
@Override
|
|
|
public Response run() throws IOException {
|
|
|
- return post(in, ugi, delegation, nnRpcAddr, path.getAbsolutePath(), op,
|
|
|
- bufferSize);
|
|
|
+ return post(in, nnId, path.getAbsolutePath(), op,
|
|
|
+ bufferSize);
|
|
|
}
|
|
|
});
|
|
|
}
|
|
|
|
|
|
private Response post(
|
|
|
final InputStream in,
|
|
|
- final UserGroupInformation ugi,
|
|
|
- final DelegationParam delegation,
|
|
|
- final InetSocketAddress nnRpcAddr,
|
|
|
+ final String nnId,
|
|
|
final String fullpath,
|
|
|
final PostOpParam op,
|
|
|
final BufferSizeParam bufferSize
|
|
@@ -304,7 +323,7 @@ public class DatanodeWebHdfsMethods {
|
|
|
{
|
|
|
final Configuration conf = new Configuration(datanode.getConf());
|
|
|
final int b = bufferSize.getValue(conf);
|
|
|
- DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
|
|
|
+ DFSClient dfsclient = newDfsClient(nnId, conf);
|
|
|
FSDataOutputStream out = null;
|
|
|
try {
|
|
|
out = dfsclient.append(fullpath, b, null, null);
|
|
@@ -332,9 +351,9 @@ public class DatanodeWebHdfsMethods {
|
|
|
@Context final UserGroupInformation ugi,
|
|
|
@QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
|
|
|
final DelegationParam delegation,
|
|
|
- @QueryParam(NamenodeRpcAddressParam.NAME)
|
|
|
- @DefaultValue(NamenodeRpcAddressParam.DEFAULT)
|
|
|
- final NamenodeRpcAddressParam namenodeRpcAddress,
|
|
|
+ @QueryParam(NamenodeAddressParam.NAME)
|
|
|
+ @DefaultValue(NamenodeAddressParam.DEFAULT)
|
|
|
+ final NamenodeAddressParam namenode,
|
|
|
@QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT)
|
|
|
final GetOpParam op,
|
|
|
@QueryParam(OffsetParam.NAME) @DefaultValue(OffsetParam.DEFAULT)
|
|
@@ -344,7 +363,7 @@ public class DatanodeWebHdfsMethods {
|
|
|
@QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
|
|
|
final BufferSizeParam bufferSize
|
|
|
) throws IOException, InterruptedException {
|
|
|
- return get(ugi, delegation, namenodeRpcAddress, ROOT, op, offset, length,
|
|
|
+ return get(ugi, delegation, namenode, ROOT, op, offset, length,
|
|
|
bufferSize);
|
|
|
}
|
|
|
|
|
@@ -356,9 +375,9 @@ public class DatanodeWebHdfsMethods {
|
|
|
@Context final UserGroupInformation ugi,
|
|
|
@QueryParam(DelegationParam.NAME) @DefaultValue(DelegationParam.DEFAULT)
|
|
|
final DelegationParam delegation,
|
|
|
- @QueryParam(NamenodeRpcAddressParam.NAME)
|
|
|
- @DefaultValue(NamenodeRpcAddressParam.DEFAULT)
|
|
|
- final NamenodeRpcAddressParam namenodeRpcAddress,
|
|
|
+ @QueryParam(NamenodeAddressParam.NAME)
|
|
|
+ @DefaultValue(NamenodeAddressParam.DEFAULT)
|
|
|
+ final NamenodeAddressParam namenode,
|
|
|
@PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
|
|
|
@QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT)
|
|
|
final GetOpParam op,
|
|
@@ -370,22 +389,20 @@ public class DatanodeWebHdfsMethods {
|
|
|
final BufferSizeParam bufferSize
|
|
|
) throws IOException, InterruptedException {
|
|
|
|
|
|
- final InetSocketAddress nnRpcAddr = namenodeRpcAddress.getValue();
|
|
|
- init(ugi, delegation, nnRpcAddr, path, op, offset, length, bufferSize);
|
|
|
+ final String nnId = namenode.getValue();
|
|
|
+ init(ugi, delegation, nnId, path, op, offset, length, bufferSize);
|
|
|
|
|
|
return ugi.doAs(new PrivilegedExceptionAction<Response>() {
|
|
|
@Override
|
|
|
public Response run() throws IOException {
|
|
|
- return get(ugi, delegation, nnRpcAddr, path.getAbsolutePath(), op,
|
|
|
- offset, length, bufferSize);
|
|
|
+ return get(nnId, path.getAbsolutePath(), op, offset,
|
|
|
+ length, bufferSize);
|
|
|
}
|
|
|
});
|
|
|
}
|
|
|
|
|
|
private Response get(
|
|
|
- final UserGroupInformation ugi,
|
|
|
- final DelegationParam delegation,
|
|
|
- final InetSocketAddress nnRpcAddr,
|
|
|
+ final String nnId,
|
|
|
final String fullpath,
|
|
|
final GetOpParam op,
|
|
|
final OffsetParam offset,
|
|
@@ -399,7 +416,7 @@ public class DatanodeWebHdfsMethods {
|
|
|
case OPEN:
|
|
|
{
|
|
|
final int b = bufferSize.getValue(conf);
|
|
|
- final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
|
|
|
+ final DFSClient dfsclient = newDfsClient(nnId, conf);
|
|
|
HdfsDataInputStream in = null;
|
|
|
try {
|
|
|
in = new HdfsDataInputStream(dfsclient.open(fullpath, b, true));
|
|
@@ -426,7 +443,7 @@ public class DatanodeWebHdfsMethods {
|
|
|
case GETFILECHECKSUM:
|
|
|
{
|
|
|
MD5MD5CRC32FileChecksum checksum = null;
|
|
|
- DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
|
|
|
+ DFSClient dfsclient = newDfsClient(nnId, conf);
|
|
|
try {
|
|
|
checksum = dfsclient.getFileChecksum(fullpath);
|
|
|
dfsclient.close();
|
|
@@ -441,4 +458,10 @@ public class DatanodeWebHdfsMethods {
|
|
|
throw new UnsupportedOperationException(op + " is not supported");
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private static DFSClient newDfsClient(String nnId,
|
|
|
+ Configuration conf) throws IOException {
|
|
|
+ URI uri = URI.create(HdfsConstants.HDFS_URI_SCHEME + "://" + nnId);
|
|
|
+ return new DFSClient(uri, conf);
|
|
|
+ }
|
|
|
}
|