|
@@ -27,6 +27,7 @@ import java.net.URI;
|
|
|
import java.net.URISyntaxException;
|
|
|
import java.security.PrivilegedExceptionAction;
|
|
|
import java.util.EnumSet;
|
|
|
+import java.util.HashSet;
|
|
|
import java.util.List;
|
|
|
|
|
|
import javax.servlet.ServletContext;
|
|
@@ -82,6 +83,7 @@ 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.ExcludeDatanodesParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.GetOpParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.GroupParam;
|
|
|
import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
|
|
@@ -111,11 +113,13 @@ import org.apache.hadoop.hdfs.web.resources.XAttrValueParam;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.ipc.Server;
|
|
|
import org.apache.hadoop.net.NetworkTopology.InvalidTopologyException;
|
|
|
+import org.apache.hadoop.net.Node;
|
|
|
import org.apache.hadoop.net.NodeBase;
|
|
|
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;
|
|
|
+import org.apache.hadoop.util.StringUtils;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Charsets;
|
|
@@ -188,12 +192,26 @@ public class NamenodeWebHdfsMethods {
|
|
|
}
|
|
|
return np;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
static DatanodeInfo chooseDatanode(final NameNode namenode,
|
|
|
final String path, final HttpOpParam.Op op, final long openOffset,
|
|
|
- final long blocksize) throws IOException {
|
|
|
+ final long blocksize, final String excludeDatanodes) throws IOException {
|
|
|
final BlockManager bm = namenode.getNamesystem().getBlockManager();
|
|
|
+
|
|
|
+ HashSet<Node> excludes = new HashSet<Node>();
|
|
|
+ if (excludeDatanodes != null) {
|
|
|
+ for (String host : StringUtils
|
|
|
+ .getTrimmedStringCollection(excludeDatanodes)) {
|
|
|
+ int idx = host.indexOf(":");
|
|
|
+ if (idx != -1) {
|
|
|
+ excludes.add(bm.getDatanodeManager().getDatanodeByXferAddr(
|
|
|
+ host.substring(0, idx), Integer.parseInt(host.substring(idx + 1))));
|
|
|
+ } else {
|
|
|
+ excludes.add(bm.getDatanodeManager().getDatanodeByHost(host));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
if (op == PutOpParam.Op.CREATE) {
|
|
|
//choose a datanode near to client
|
|
@@ -201,7 +219,7 @@ public class NamenodeWebHdfsMethods {
|
|
|
).getDatanodeByHost(getRemoteAddress());
|
|
|
if (clientNode != null) {
|
|
|
final DatanodeStorageInfo[] storages = bm.chooseTarget4WebHDFS(
|
|
|
- path, clientNode, blocksize);
|
|
|
+ path, clientNode, excludes, blocksize);
|
|
|
if (storages.length > 0) {
|
|
|
return storages[0].getDatanodeDescriptor();
|
|
|
}
|
|
@@ -228,7 +246,7 @@ public class NamenodeWebHdfsMethods {
|
|
|
final LocatedBlocks locations = np.getBlockLocations(path, offset, 1);
|
|
|
final int count = locations.locatedBlockCount();
|
|
|
if (count > 0) {
|
|
|
- return bestNode(locations.get(0).getLocations());
|
|
|
+ return bestNode(locations.get(0).getLocations(), excludes);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -242,11 +260,14 @@ public class NamenodeWebHdfsMethods {
|
|
|
* sorted based on availability and network distances, thus it is sufficient
|
|
|
* to return the first element of the node here.
|
|
|
*/
|
|
|
- private static DatanodeInfo bestNode(DatanodeInfo[] nodes) throws IOException {
|
|
|
- if (nodes.length == 0 || nodes[0].isDecommissioned()) {
|
|
|
- throw new IOException("No active nodes contain this block");
|
|
|
+ private static DatanodeInfo bestNode(DatanodeInfo[] nodes,
|
|
|
+ HashSet<Node> excludes) throws IOException {
|
|
|
+ for (DatanodeInfo dn: nodes) {
|
|
|
+ if (false == dn.isDecommissioned() && false == excludes.contains(dn)) {
|
|
|
+ return dn;
|
|
|
+ }
|
|
|
}
|
|
|
- return nodes[0];
|
|
|
+ throw new IOException("No active nodes contain this block");
|
|
|
}
|
|
|
|
|
|
private Token<? extends TokenIdentifier> generateDelegationToken(
|
|
@@ -265,11 +286,12 @@ public class NamenodeWebHdfsMethods {
|
|
|
final UserGroupInformation ugi, final DelegationParam delegation,
|
|
|
final UserParam username, final DoAsParam doAsUser,
|
|
|
final String path, final HttpOpParam.Op op, final long openOffset,
|
|
|
- final long blocksize,
|
|
|
+ final long blocksize, final String excludeDatanodes,
|
|
|
final Param<?, ?>... parameters) throws URISyntaxException, IOException {
|
|
|
final DatanodeInfo dn;
|
|
|
try {
|
|
|
- dn = chooseDatanode(namenode, path, op, openOffset, blocksize);
|
|
|
+ dn = chooseDatanode(namenode, path, op, openOffset, blocksize,
|
|
|
+ excludeDatanodes);
|
|
|
} catch (InvalidTopologyException ite) {
|
|
|
throw new IOException("Failed to find datanode, suggest to check cluster health.", ite);
|
|
|
}
|
|
@@ -356,13 +378,15 @@ public class NamenodeWebHdfsMethods {
|
|
|
@QueryParam(SnapshotNameParam.NAME) @DefaultValue(SnapshotNameParam.DEFAULT)
|
|
|
final SnapshotNameParam snapshotName,
|
|
|
@QueryParam(OldSnapshotNameParam.NAME) @DefaultValue(OldSnapshotNameParam.DEFAULT)
|
|
|
- final OldSnapshotNameParam oldSnapshotName
|
|
|
- )throws IOException, InterruptedException {
|
|
|
+ final OldSnapshotNameParam oldSnapshotName,
|
|
|
+ @QueryParam(ExcludeDatanodesParam.NAME) @DefaultValue(ExcludeDatanodesParam.DEFAULT)
|
|
|
+ final ExcludeDatanodesParam excludeDatanodes
|
|
|
+ ) throws IOException, InterruptedException {
|
|
|
return put(ugi, delegation, username, doAsUser, ROOT, op, destination,
|
|
|
owner, group, permission, overwrite, bufferSize, replication,
|
|
|
blockSize, modificationTime, accessTime, renameOptions, createParent,
|
|
|
delegationTokenArgument, aclPermission, xattrName, xattrValue,
|
|
|
- xattrSetFlag, snapshotName, oldSnapshotName);
|
|
|
+ xattrSetFlag, snapshotName, oldSnapshotName, excludeDatanodes);
|
|
|
}
|
|
|
|
|
|
/** Handle HTTP PUT request. */
|
|
@@ -418,14 +442,16 @@ public class NamenodeWebHdfsMethods {
|
|
|
@QueryParam(SnapshotNameParam.NAME) @DefaultValue(SnapshotNameParam.DEFAULT)
|
|
|
final SnapshotNameParam snapshotName,
|
|
|
@QueryParam(OldSnapshotNameParam.NAME) @DefaultValue(OldSnapshotNameParam.DEFAULT)
|
|
|
- final OldSnapshotNameParam oldSnapshotName
|
|
|
+ final OldSnapshotNameParam oldSnapshotName,
|
|
|
+ @QueryParam(ExcludeDatanodesParam.NAME) @DefaultValue(ExcludeDatanodesParam.DEFAULT)
|
|
|
+ final ExcludeDatanodesParam excludeDatanodes
|
|
|
) throws IOException, InterruptedException {
|
|
|
|
|
|
init(ugi, delegation, username, doAsUser, path, op, destination, owner,
|
|
|
group, permission, overwrite, bufferSize, replication, blockSize,
|
|
|
modificationTime, accessTime, renameOptions, delegationTokenArgument,
|
|
|
aclPermission, xattrName, xattrValue, xattrSetFlag, snapshotName,
|
|
|
- oldSnapshotName);
|
|
|
+ oldSnapshotName, excludeDatanodes);
|
|
|
|
|
|
return ugi.doAs(new PrivilegedExceptionAction<Response>() {
|
|
|
@Override
|
|
@@ -436,7 +462,7 @@ public class NamenodeWebHdfsMethods {
|
|
|
permission, overwrite, bufferSize, replication, blockSize,
|
|
|
modificationTime, accessTime, renameOptions, createParent,
|
|
|
delegationTokenArgument, aclPermission, xattrName, xattrValue,
|
|
|
- xattrSetFlag, snapshotName, oldSnapshotName);
|
|
|
+ xattrSetFlag, snapshotName, oldSnapshotName, excludeDatanodes);
|
|
|
} finally {
|
|
|
reset();
|
|
|
}
|
|
@@ -469,7 +495,8 @@ public class NamenodeWebHdfsMethods {
|
|
|
final XAttrValueParam xattrValue,
|
|
|
final XAttrSetFlagParam xattrSetFlag,
|
|
|
final SnapshotNameParam snapshotName,
|
|
|
- final OldSnapshotNameParam oldSnapshotName
|
|
|
+ final OldSnapshotNameParam oldSnapshotName,
|
|
|
+ final ExcludeDatanodesParam exclDatanodes
|
|
|
) throws IOException, URISyntaxException {
|
|
|
|
|
|
final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
|
|
@@ -479,9 +506,10 @@ public class NamenodeWebHdfsMethods {
|
|
|
switch(op.getValue()) {
|
|
|
case CREATE:
|
|
|
{
|
|
|
- final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser,
|
|
|
- fullpath, op.getValue(), -1L, blockSize.getValue(conf),
|
|
|
- permission, overwrite, bufferSize, replication, blockSize);
|
|
|
+ final URI uri = redirectURI(namenode, ugi, delegation, username,
|
|
|
+ doAsUser, fullpath, op.getValue(), -1L, blockSize.getValue(conf),
|
|
|
+ exclDatanodes.getValue(), permission, overwrite, bufferSize,
|
|
|
+ replication, blockSize);
|
|
|
return Response.temporaryRedirect(uri).type(MediaType.APPLICATION_OCTET_STREAM).build();
|
|
|
}
|
|
|
case MKDIRS:
|
|
@@ -614,9 +642,12 @@ public class NamenodeWebHdfsMethods {
|
|
|
@QueryParam(ConcatSourcesParam.NAME) @DefaultValue(ConcatSourcesParam.DEFAULT)
|
|
|
final ConcatSourcesParam concatSrcs,
|
|
|
@QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
|
|
|
- final BufferSizeParam bufferSize
|
|
|
+ final BufferSizeParam bufferSize,
|
|
|
+ @QueryParam(ExcludeDatanodesParam.NAME) @DefaultValue(ExcludeDatanodesParam.DEFAULT)
|
|
|
+ final ExcludeDatanodesParam excludeDatanodes
|
|
|
) throws IOException, InterruptedException {
|
|
|
- return post(ugi, delegation, username, doAsUser, ROOT, op, concatSrcs, bufferSize);
|
|
|
+ return post(ugi, delegation, username, doAsUser, ROOT, op, concatSrcs,
|
|
|
+ bufferSize, excludeDatanodes);
|
|
|
}
|
|
|
|
|
|
/** Handle HTTP POST request. */
|
|
@@ -638,17 +669,21 @@ public class NamenodeWebHdfsMethods {
|
|
|
@QueryParam(ConcatSourcesParam.NAME) @DefaultValue(ConcatSourcesParam.DEFAULT)
|
|
|
final ConcatSourcesParam concatSrcs,
|
|
|
@QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
|
|
|
- final BufferSizeParam bufferSize
|
|
|
+ final BufferSizeParam bufferSize,
|
|
|
+ @QueryParam(ExcludeDatanodesParam.NAME) @DefaultValue(ExcludeDatanodesParam.DEFAULT)
|
|
|
+ final ExcludeDatanodesParam excludeDatanodes
|
|
|
) throws IOException, InterruptedException {
|
|
|
|
|
|
- init(ugi, delegation, username, doAsUser, path, op, concatSrcs, bufferSize);
|
|
|
+ init(ugi, delegation, username, doAsUser, path, op, concatSrcs, bufferSize,
|
|
|
+ excludeDatanodes);
|
|
|
|
|
|
return ugi.doAs(new PrivilegedExceptionAction<Response>() {
|
|
|
@Override
|
|
|
public Response run() throws IOException, URISyntaxException {
|
|
|
try {
|
|
|
return post(ugi, delegation, username, doAsUser,
|
|
|
- path.getAbsolutePath(), op, concatSrcs, bufferSize);
|
|
|
+ path.getAbsolutePath(), op, concatSrcs, bufferSize,
|
|
|
+ excludeDatanodes);
|
|
|
} finally {
|
|
|
reset();
|
|
|
}
|
|
@@ -664,15 +699,17 @@ public class NamenodeWebHdfsMethods {
|
|
|
final String fullpath,
|
|
|
final PostOpParam op,
|
|
|
final ConcatSourcesParam concatSrcs,
|
|
|
- final BufferSizeParam bufferSize
|
|
|
+ final BufferSizeParam bufferSize,
|
|
|
+ final ExcludeDatanodesParam excludeDatanodes
|
|
|
) throws IOException, URISyntaxException {
|
|
|
final NameNode namenode = (NameNode)context.getAttribute("name.node");
|
|
|
|
|
|
switch(op.getValue()) {
|
|
|
case APPEND:
|
|
|
{
|
|
|
- final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser,
|
|
|
- fullpath, op.getValue(), -1L, -1L, bufferSize);
|
|
|
+ final URI uri = redirectURI(namenode, ugi, delegation, username,
|
|
|
+ doAsUser, fullpath, op.getValue(), -1L, -1L,
|
|
|
+ excludeDatanodes.getValue(), bufferSize);
|
|
|
return Response.temporaryRedirect(uri).type(MediaType.APPLICATION_OCTET_STREAM).build();
|
|
|
}
|
|
|
case CONCAT:
|
|
@@ -710,10 +747,12 @@ public class NamenodeWebHdfsMethods {
|
|
|
@QueryParam(XAttrNameParam.NAME) @DefaultValue(XAttrNameParam.DEFAULT)
|
|
|
final List<XAttrNameParam> xattrNames,
|
|
|
@QueryParam(XAttrEncodingParam.NAME) @DefaultValue(XAttrEncodingParam.DEFAULT)
|
|
|
- final XAttrEncodingParam xattrEncoding
|
|
|
+ final XAttrEncodingParam xattrEncoding,
|
|
|
+ @QueryParam(ExcludeDatanodesParam.NAME) @DefaultValue(ExcludeDatanodesParam.DEFAULT)
|
|
|
+ final ExcludeDatanodesParam excludeDatanodes
|
|
|
) throws IOException, InterruptedException {
|
|
|
return get(ugi, delegation, username, doAsUser, ROOT, op, offset, length,
|
|
|
- renewer, bufferSize, xattrNames, xattrEncoding);
|
|
|
+ renewer, bufferSize, xattrNames, xattrEncoding, excludeDatanodes);
|
|
|
}
|
|
|
|
|
|
/** Handle HTTP GET request. */
|
|
@@ -742,11 +781,13 @@ public class NamenodeWebHdfsMethods {
|
|
|
@QueryParam(XAttrNameParam.NAME) @DefaultValue(XAttrNameParam.DEFAULT)
|
|
|
final List<XAttrNameParam> xattrNames,
|
|
|
@QueryParam(XAttrEncodingParam.NAME) @DefaultValue(XAttrEncodingParam.DEFAULT)
|
|
|
- final XAttrEncodingParam xattrEncoding
|
|
|
+ final XAttrEncodingParam xattrEncoding,
|
|
|
+ @QueryParam(ExcludeDatanodesParam.NAME) @DefaultValue(ExcludeDatanodesParam.DEFAULT)
|
|
|
+ final ExcludeDatanodesParam excludeDatanodes
|
|
|
) throws IOException, InterruptedException {
|
|
|
|
|
|
init(ugi, delegation, username, doAsUser, path, op, offset, length,
|
|
|
- renewer, bufferSize, xattrEncoding);
|
|
|
+ renewer, bufferSize, xattrEncoding, excludeDatanodes);
|
|
|
|
|
|
return ugi.doAs(new PrivilegedExceptionAction<Response>() {
|
|
|
@Override
|
|
@@ -754,7 +795,7 @@ public class NamenodeWebHdfsMethods {
|
|
|
try {
|
|
|
return get(ugi, delegation, username, doAsUser,
|
|
|
path.getAbsolutePath(), op, offset, length, renewer, bufferSize,
|
|
|
- xattrNames, xattrEncoding);
|
|
|
+ xattrNames, xattrEncoding, excludeDatanodes);
|
|
|
} finally {
|
|
|
reset();
|
|
|
}
|
|
@@ -774,7 +815,8 @@ public class NamenodeWebHdfsMethods {
|
|
|
final RenewerParam renewer,
|
|
|
final BufferSizeParam bufferSize,
|
|
|
final List<XAttrNameParam> xattrNames,
|
|
|
- final XAttrEncodingParam xattrEncoding
|
|
|
+ final XAttrEncodingParam xattrEncoding,
|
|
|
+ final ExcludeDatanodesParam excludeDatanodes
|
|
|
) throws IOException, URISyntaxException {
|
|
|
final NameNode namenode = (NameNode)context.getAttribute("name.node");
|
|
|
final NamenodeProtocols np = getRPCServer(namenode);
|
|
@@ -782,8 +824,9 @@ public class NamenodeWebHdfsMethods {
|
|
|
switch(op.getValue()) {
|
|
|
case OPEN:
|
|
|
{
|
|
|
- final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser,
|
|
|
- fullpath, op.getValue(), offset.getValue(), -1L, offset, length, bufferSize);
|
|
|
+ final URI uri = redirectURI(namenode, ugi, delegation, username,
|
|
|
+ doAsUser, fullpath, op.getValue(), offset.getValue(), -1L,
|
|
|
+ excludeDatanodes.getValue(), offset, length, bufferSize);
|
|
|
return Response.temporaryRedirect(uri).type(MediaType.APPLICATION_OCTET_STREAM).build();
|
|
|
}
|
|
|
case GET_BLOCK_LOCATIONS:
|
|
@@ -819,7 +862,7 @@ public class NamenodeWebHdfsMethods {
|
|
|
case GETFILECHECKSUM:
|
|
|
{
|
|
|
final URI uri = redirectURI(namenode, ugi, delegation, username, doAsUser,
|
|
|
- fullpath, op.getValue(), -1L, -1L);
|
|
|
+ fullpath, op.getValue(), -1L, -1L, null);
|
|
|
return Response.temporaryRedirect(uri).type(MediaType.APPLICATION_OCTET_STREAM).build();
|
|
|
}
|
|
|
case GETDELEGATIONTOKEN:
|