HDFS-2284. Add a new FileSystem, webhdfs://, for supporting write Http access to HDFS.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1167662 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2011-09-11 01:41:42 +00:00
parent a3df5d0b26
commit 61d0b7530c
44 changed files with 2962 additions and 8 deletions

View File

@ -5,6 +5,9 @@ Trunk (unreleased changes)
HDFS-395. DFS Scalability: Incremental block reports. (Tomasz Nykiel
via hairong)
HDFS-2284. Add a new FileSystem, webhdfs://, for supporting write Http
access to HDFS. (szetszwo)
IMPROVEMENTS
HADOOP-7524 Change RPC to allow multiple protocols including multuple versions of the same protocol (sanjay Radia)

View File

@ -802,7 +802,7 @@ private DFSOutputStream callAppend(HdfsFileStatus stat, String src,
*
* @see ClientProtocol#append(String, String)
*/
DFSOutputStream append(String src, int buffersize, Progressable progress)
public DFSOutputStream append(String src, int buffersize, Progressable progress)
throws IOException {
checkOpen();
HdfsFileStatus stat = getFileInfo(src);

View File

@ -98,7 +98,7 @@
* datanode from the original pipeline. The DataStreamer now
* starts sending packets from the dataQueue.
****************************************************************/
class DFSOutputStream extends FSOutputSummer implements Syncable {
public class DFSOutputStream extends FSOutputSummer implements Syncable {
private final DFSClient dfsClient;
private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB
private Socket s;
@ -1707,7 +1707,7 @@ synchronized void setTestFilename(String newname) {
/**
* Returns the size of a file as it was when this stream was opened
*/
long getInitialLen() {
public long getInitialLen() {
return initialFileSize;
}

View File

@ -250,7 +250,7 @@ public URI getUri() {
* @return namenode URL referring to the given path
* @throws IOException on error constructing the URL
*/
URL getNamenodeURL(String path, String query) throws IOException {
protected URL getNamenodeURL(String path, String query) throws IOException {
final URL url = new URL("http", nnAddr.getHostName(),
nnAddr.getPort(), path + '?' + query);
if (LOG.isTraceEnabled()) {
@ -317,6 +317,7 @@ protected String addDelegationTokenParam(String query) throws IOException {
@Override
public FSDataInputStream open(Path f, int buffersize) throws IOException {
f = f.makeQualified(getUri(), getWorkingDirectory());
String path = "/data" + ServletUtil.encodePath(f.toUri().getPath());
String query = addDelegationTokenParam("ugi=" + getEncodedUgiParameter());
URL u = getNamenodeURL(path, query);

View File

@ -18,7 +18,50 @@
package org.apache.hadoop.hdfs.server.datanode;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PLUGINS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STARTUP_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STORAGEID_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_TRANSFERTO_ALLOWED_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_FEDERATION_NAMESERVICES;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import java.io.BufferedOutputStream;
@ -93,9 +136,11 @@
import org.apache.hadoop.hdfs.server.datanode.FSDataset.VolumeInfo;
import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
import org.apache.hadoop.hdfs.server.datanode.web.resources.DatanodeWebHdfsMethods;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets;
import org.apache.hadoop.hdfs.server.namenode.StreamFile;
import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
@ -109,7 +154,8 @@
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
import org.apache.hadoop.hdfs.web.resources.Param;
import org.apache.hadoop.http.HttpServer;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.ProtocolSignature;
@ -502,6 +548,11 @@ conf, new AccessControlList(conf.get(DFS_ADMIN, " ")),
this.infoServer.setAttribute(JspHelper.CURRENT_CONF, conf);
this.infoServer.addServlet(null, "/blockScannerReport",
DataBlockScanner.Servlet.class);
infoServer.addJerseyResourcePackage(
DatanodeWebHdfsMethods.class.getPackage().getName()
+ ";" + Param.class.getPackage().getName(),
"/" + WebHdfsFileSystem.PATH_PREFIX + "/*");
this.infoServer.start();
}

View File

@ -0,0 +1,169 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
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.util.EnumSet;
import javax.servlet.ServletContext;
import javax.ws.rs.Consumes;
import javax.ws.rs.DefaultValue;
import javax.ws.rs.POST;
import javax.ws.rs.PUT;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSOutputStream;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
import org.apache.hadoop.hdfs.web.resources.OverwriteParam;
import org.apache.hadoop.hdfs.web.resources.Param;
import org.apache.hadoop.hdfs.web.resources.PermissionParam;
import org.apache.hadoop.hdfs.web.resources.PostOpParam;
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;
/** Web-hdfs DataNode implementation. */
@Path("")
public class DatanodeWebHdfsMethods {
private static final Log LOG = LogFactory.getLog(DatanodeWebHdfsMethods.class);
private @Context ServletContext context;
/** Handle HTTP PUT request. */
@PUT
@Path("{" + UriFsPathParam.NAME + ":.*}")
@Consumes({"*/*"})
@Produces({MediaType.APPLICATION_JSON})
public Response put(
final InputStream in,
@PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
@QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT)
final PutOpParam op,
@QueryParam(PermissionParam.NAME) @DefaultValue(PermissionParam.DEFAULT)
final PermissionParam permission,
@QueryParam(OverwriteParam.NAME) @DefaultValue(OverwriteParam.DEFAULT)
final OverwriteParam overwrite,
@QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
final BufferSizeParam bufferSize,
@QueryParam(ReplicationParam.NAME) @DefaultValue(ReplicationParam.DEFAULT)
final ReplicationParam replication,
@QueryParam(BlockSizeParam.NAME) @DefaultValue(BlockSizeParam.DEFAULT)
final BlockSizeParam blockSize
) throws IOException, URISyntaxException {
if (LOG.isTraceEnabled()) {
LOG.trace(op + ": " + path
+ Param.toSortedString(", ", permission, overwrite, bufferSize,
replication, blockSize));
}
final String fullpath = path.getAbsolutePath();
final DataNode datanode = (DataNode)context.getAttribute("datanode");
switch(op.getValue()) {
case CREATE:
{
final Configuration conf = new Configuration(datanode.getConf());
final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf);
final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
final FSDataOutputStream out = new FSDataOutputStream(dfsclient.create(
fullpath, permission.getFsPermission(),
overwrite.getValue() ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
: EnumSet.of(CreateFlag.CREATE),
replication.getValue(), blockSize.getValue(), null,
bufferSize.getValue()), null);
try {
IOUtils.copyBytes(in, out, bufferSize.getValue());
} finally {
out.close();
}
final InetSocketAddress nnHttpAddr = NameNode.getHttpAddress(conf);
final URI uri = new URI(WebHdfsFileSystem.SCHEME, null,
nnHttpAddr.getHostName(), nnHttpAddr.getPort(), fullpath, null, null);
return Response.created(uri).type(MediaType.APPLICATION_JSON).build();
}
default:
throw new UnsupportedOperationException(op + " is not supported");
}
}
/** Handle HTTP POST request. */
@POST
@Path("{" + UriFsPathParam.NAME + ":.*}")
@Consumes({"*/*"})
@Produces({MediaType.APPLICATION_JSON})
public Response post(
final InputStream in,
@PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
@QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT)
final PostOpParam op,
@QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
final BufferSizeParam bufferSize
) throws IOException, URISyntaxException {
if (LOG.isTraceEnabled()) {
LOG.trace(op + ": " + path
+ Param.toSortedString(", ", bufferSize));
}
final String fullpath = path.getAbsolutePath();
final DataNode datanode = (DataNode)context.getAttribute("datanode");
switch(op.getValue()) {
case APPEND:
{
final Configuration conf = new Configuration(datanode.getConf());
final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf);
final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
final DFSOutputStream dfsout = dfsclient.append(fullpath,
bufferSize.getValue(), null);
final FSDataOutputStream out = new FSDataOutputStream(dfsout, null,
dfsout.getInitialLen());
try {
IOUtils.copyBytes(in, out, bufferSize.getValue());
} finally {
out.close();
}
return Response.ok().type(MediaType.APPLICATION_JSON).build();
}
default:
throw new UnsupportedOperationException(op + " is not supported");
}
}
}

View File

@ -279,6 +279,11 @@ protected void setRpcServerAddress(Configuration conf,
}
protected InetSocketAddress getHttpServerAddress(Configuration conf) {
return getHttpAddress(conf);
}
/** @return the NameNode HTTP address set in the conf. */
public static InetSocketAddress getHttpAddress(Configuration conf) {
return NetUtils.createSocketAddr(
conf.get(DFS_NAMENODE_HTTP_ADDRESS_KEY, DFS_NAMENODE_HTTP_ADDRESS_DEFAULT));
}

View File

@ -24,18 +24,20 @@
import javax.servlet.ServletContext;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
import org.apache.hadoop.hdfs.web.resources.Param;
import org.apache.hadoop.http.HttpServer;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.classification.InterfaceAudience;
/**
* Encapsulates the HTTP server started by the NameNode.
*/
@ -179,6 +181,11 @@ private static void setupServlets(HttpServer httpServer) {
FileChecksumServlets.RedirectServlet.class, false);
httpServer.addInternalServlet("contentSummary", "/contentSummary/*",
ContentSummaryServlet.class, false);
httpServer.addJerseyResourcePackage(
NamenodeWebHdfsMethods.class.getPackage().getName()
+ ";" + Param.class.getPackage().getName(),
"/" + WebHdfsFileSystem.PATH_PREFIX + "/*");
}
public static FSImage getFsImageFromContext(ServletContext context) {

View File

@ -0,0 +1,315 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.namenode.web.resources;
import java.io.IOException;
import java.io.InputStream;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.EnumSet;
import javax.servlet.ServletContext;
import javax.ws.rs.Consumes;
import javax.ws.rs.DELETE;
import javax.ws.rs.DefaultValue;
import javax.ws.rs.GET;
import javax.ws.rs.POST;
import javax.ws.rs.PUT;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.hdfs.web.JsonUtil;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
import org.apache.hadoop.hdfs.web.resources.DeleteOpParam;
import org.apache.hadoop.hdfs.web.resources.DstPathParam;
import org.apache.hadoop.hdfs.web.resources.GetOpParam;
import org.apache.hadoop.hdfs.web.resources.GroupParam;
import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
import org.apache.hadoop.hdfs.web.resources.ModificationTimeParam;
import org.apache.hadoop.hdfs.web.resources.OverwriteParam;
import org.apache.hadoop.hdfs.web.resources.OwnerParam;
import org.apache.hadoop.hdfs.web.resources.Param;
import org.apache.hadoop.hdfs.web.resources.PermissionParam;
import org.apache.hadoop.hdfs.web.resources.PostOpParam;
import org.apache.hadoop.hdfs.web.resources.PutOpParam;
import org.apache.hadoop.hdfs.web.resources.RecursiveParam;
import org.apache.hadoop.hdfs.web.resources.RenameOptionSetParam;
import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
import org.apache.hadoop.hdfs.web.resources.UriFsPathParam;
import org.apache.hadoop.net.NodeBase;
/** Web-hdfs NameNode implementation. */
@Path("")
public class NamenodeWebHdfsMethods {
private static final Log LOG = LogFactory.getLog(NamenodeWebHdfsMethods.class);
private @Context ServletContext context;
private static DatanodeInfo chooseDatanode(final NameNode namenode,
final String path, final HttpOpParam.Op op) throws IOException {
if (op == PostOpParam.Op.APPEND) {
final HdfsFileStatus status = namenode.getRpcServer().getFileInfo(path);
final long len = status.getLen();
if (len > 0) {
final LocatedBlocks locations = namenode.getRpcServer().getBlockLocations(path, len-1, 1);
final int count = locations.locatedBlockCount();
if (count > 0) {
return JspHelper.bestNode(locations.get(count - 1));
}
}
}
return (DatanodeDescriptor)namenode.getNamesystem().getBlockManager(
).getDatanodeManager().getNetworkTopology().chooseRandom(
NodeBase.ROOT);
}
private static URI redirectURI(final NameNode namenode,
final String path, final HttpOpParam.Op op,
final Param<?, ?>... parameters) throws URISyntaxException, IOException {
final DatanodeInfo dn = chooseDatanode(namenode, path, op);
final String query = op.toQueryString() + Param.toSortedString("&", parameters);
final String uripath = "/" + WebHdfsFileSystem.PATH_PREFIX + path;
final URI uri = new URI("http", null, dn.getHostName(), dn.getInfoPort(),
uripath, query, null);
if (LOG.isTraceEnabled()) {
LOG.trace("redirectURI=" + uri);
}
return uri;
}
/** Handle HTTP PUT request. */
@PUT
@Path("{" + UriFsPathParam.NAME + ":.*}")
@Consumes({"*/*"})
@Produces({MediaType.APPLICATION_JSON})
public Response put(
final InputStream in,
@PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
@QueryParam(PutOpParam.NAME) @DefaultValue(PutOpParam.DEFAULT)
final PutOpParam op,
@QueryParam(DstPathParam.NAME) @DefaultValue(DstPathParam.DEFAULT)
final DstPathParam dstPath,
@QueryParam(OwnerParam.NAME) @DefaultValue(OwnerParam.DEFAULT)
final OwnerParam owner,
@QueryParam(GroupParam.NAME) @DefaultValue(GroupParam.DEFAULT)
final GroupParam group,
@QueryParam(PermissionParam.NAME) @DefaultValue(PermissionParam.DEFAULT)
final PermissionParam permission,
@QueryParam(OverwriteParam.NAME) @DefaultValue(OverwriteParam.DEFAULT)
final OverwriteParam overwrite,
@QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
final BufferSizeParam bufferSize,
@QueryParam(ReplicationParam.NAME) @DefaultValue(ReplicationParam.DEFAULT)
final ReplicationParam replication,
@QueryParam(BlockSizeParam.NAME) @DefaultValue(BlockSizeParam.DEFAULT)
final BlockSizeParam blockSize,
@QueryParam(ModificationTimeParam.NAME) @DefaultValue(ModificationTimeParam.DEFAULT)
final ModificationTimeParam modificationTime,
@QueryParam(AccessTimeParam.NAME) @DefaultValue(AccessTimeParam.DEFAULT)
final AccessTimeParam accessTime,
@QueryParam(RenameOptionSetParam.NAME) @DefaultValue(RenameOptionSetParam.DEFAULT)
final RenameOptionSetParam renameOptions
) throws IOException, URISyntaxException {
if (LOG.isTraceEnabled()) {
LOG.trace(op + ": " + path
+ Param.toSortedString(", ", dstPath, owner, group, permission,
overwrite, bufferSize, replication, blockSize));
}
final String fullpath = path.getAbsolutePath();
final NameNode namenode = (NameNode)context.getAttribute("name.node");
final NamenodeProtocols np = namenode.getRpcServer();
switch(op.getValue()) {
case CREATE:
{
final URI uri = redirectURI(namenode, fullpath, op.getValue(),
permission, overwrite, bufferSize, replication, blockSize);
return Response.temporaryRedirect(uri).build();
}
case MKDIRS:
{
final boolean b = np.mkdirs(fullpath, permission.getFsPermission(), true);
final String js = JsonUtil.toJsonString(PutOpParam.Op.MKDIRS, b);
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
}
case RENAME:
{
final EnumSet<Options.Rename> s = renameOptions.getValue();
if (s.isEmpty()) {
@SuppressWarnings("deprecation")
final boolean b = np.rename(fullpath, dstPath.getValue());
final String js = JsonUtil.toJsonString(PutOpParam.Op.RENAME, b);
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
} else {
np.rename(fullpath, dstPath.getValue(),
s.toArray(new Options.Rename[s.size()]));
return Response.ok().type(MediaType.APPLICATION_JSON).build();
}
}
case SETREPLICATION:
{
final boolean b = np.setReplication(fullpath, replication.getValue());
final String js = JsonUtil.toJsonString(PutOpParam.Op.SETREPLICATION, b);
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
}
case SETOWNER:
{
np.setOwner(fullpath, owner.getValue(), group.getValue());
return Response.ok().type(MediaType.APPLICATION_JSON).build();
}
case SETPERMISSION:
{
np.setPermission(fullpath, permission.getFsPermission());
return Response.ok().type(MediaType.APPLICATION_JSON).build();
}
case SETTIMES:
{
np.setTimes(fullpath, modificationTime.getValue(), accessTime.getValue());
return Response.ok().type(MediaType.APPLICATION_JSON).build();
}
default:
throw new UnsupportedOperationException(op + " is not supported");
}
}
/** Handle HTTP POST request. */
@POST
@Path("{" + UriFsPathParam.NAME + ":.*}")
@Consumes({"*/*"})
@Produces({MediaType.APPLICATION_JSON})
public Response post(
final InputStream in,
@PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
@QueryParam(PostOpParam.NAME) @DefaultValue(PostOpParam.DEFAULT)
final PostOpParam op,
@QueryParam(BufferSizeParam.NAME) @DefaultValue(BufferSizeParam.DEFAULT)
final BufferSizeParam bufferSize
) throws IOException, URISyntaxException {
if (LOG.isTraceEnabled()) {
LOG.trace(op + ": " + path
+ Param.toSortedString(", ", bufferSize));
}
final String fullpath = path.getAbsolutePath();
final NameNode namenode = (NameNode)context.getAttribute("name.node");
switch(op.getValue()) {
case APPEND:
{
final URI uri = redirectURI(namenode, fullpath, op.getValue(), bufferSize);
return Response.temporaryRedirect(uri).build();
}
default:
throw new UnsupportedOperationException(op + " is not supported");
}
}
private static final UriFsPathParam ROOT = new UriFsPathParam("");
/** Handle HTTP GET request for the root. */
@GET
@Path("/")
@Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON})
public Response root(
@QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT)
final GetOpParam op
) throws IOException {
return get(ROOT, op);
}
/** Handle HTTP GET request. */
@GET
@Path("{" + UriFsPathParam.NAME + ":.*}")
@Produces({MediaType.APPLICATION_OCTET_STREAM, MediaType.APPLICATION_JSON})
public Response get(
@PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
@QueryParam(GetOpParam.NAME) @DefaultValue(GetOpParam.DEFAULT)
final GetOpParam op
) throws IOException {
if (LOG.isTraceEnabled()) {
LOG.trace(op + ", " + path
+ Param.toSortedString(", "));
}
switch(op.getValue()) {
case GETFILESTATUS:
final NameNode namenode = (NameNode)context.getAttribute("name.node");
final String fullpath = path.getAbsolutePath();
final HdfsFileStatus status = namenode.getRpcServer().getFileInfo(fullpath);
final String js = JsonUtil.toJsonString(status);
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
default:
throw new UnsupportedOperationException(op + " is not supported");
}
}
/** Handle HTTP DELETE request. */
@DELETE
@Path("{path:.*}")
@Produces(MediaType.APPLICATION_JSON)
public Response delete(
@PathParam(UriFsPathParam.NAME) final UriFsPathParam path,
@QueryParam(DeleteOpParam.NAME) @DefaultValue(DeleteOpParam.DEFAULT)
final DeleteOpParam op,
@QueryParam(RecursiveParam.NAME) @DefaultValue(RecursiveParam.DEFAULT)
final RecursiveParam recursive
) throws IOException {
if (LOG.isTraceEnabled()) {
LOG.trace(op + ", " + path
+ Param.toSortedString(", ", recursive));
}
switch(op.getValue()) {
case DELETE:
final NameNode namenode = (NameNode)context.getAttribute("name.node");
final String fullpath = path.getAbsolutePath();
final boolean b = namenode.getRpcServer().delete(fullpath, recursive.getValue());
final String js = JsonUtil.toJsonString(DeleteOpParam.Op.DELETE, b);
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
default:
throw new UnsupportedOperationException(op + " is not supported");
}
}
}

View File

@ -0,0 +1,133 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web;
import java.util.Map;
import java.util.TreeMap;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.ipc.RemoteException;
import org.mortbay.util.ajax.JSON;
/** JSON Utilities */
public class JsonUtil {
private static final ThreadLocal<Map<String, Object>> jsonMap
= new ThreadLocal<Map<String, Object>>() {
@Override
protected Map<String, Object> initialValue() {
return new TreeMap<String, Object>();
}
@Override
public Map<String, Object> get() {
final Map<String, Object> m = super.get();
m.clear();
return m;
}
};
/** Convert an exception object to a Json string. */
public static String toJsonString(final Exception e) {
final Map<String, Object> m = jsonMap.get();
m.put("className", e.getClass().getName());
m.put("message", e.getMessage());
return JSON.toString(m);
}
/** Convert a Json map to a RemoteException. */
public static RemoteException toRemoteException(final Map<String, Object> m) {
final String className = (String)m.get("className");
final String message = (String)m.get("message");
return new RemoteException(className, message);
}
/** Convert a key-value pair to a Json string. */
public static String toJsonString(final Object key, final Object value) {
final Map<String, Object> m = jsonMap.get();
m.put(key instanceof String ? (String) key : key.toString(), value);
return JSON.toString(m);
}
/** Convert a FsPermission object to a string. */
public static String toString(final FsPermission permission) {
return String.format("%o", permission.toShort());
}
/** Convert a string to a FsPermission object. */
public static FsPermission toFsPermission(final String s) {
return new FsPermission(Short.parseShort(s, 8));
}
/** Convert a HdfsFileStatus object to a Json string. */
public static String toJsonString(final HdfsFileStatus status) {
final Map<String, Object> m = jsonMap.get();
if (status == null) {
m.put("isNull", true);
} else {
m.put("isNull", false);
m.put("localName", status.getLocalName());
m.put("isDir", status.isDir());
m.put("isSymlink", status.isSymlink());
if (status.isSymlink()) {
m.put("symlink", status.getSymlink());
}
m.put("len", status.getLen());
m.put("owner", status.getOwner());
m.put("group", status.getGroup());
m.put("permission", toString(status.getPermission()));
m.put("accessTime", status.getAccessTime());
m.put("modificationTime", status.getModificationTime());
m.put("blockSize", status.getBlockSize());
m.put("replication", status.getReplication());
}
return JSON.toString(m);
}
@SuppressWarnings("unchecked")
static Map<String, Object> parse(String jsonString) {
return (Map<String, Object>) JSON.parse(jsonString);
}
/** Convert a Json string to a HdfsFileStatus object. */
public static HdfsFileStatus toFileStatus(final Map<String, Object> m) {
if ((Boolean)m.get("isNull")) {
return null;
}
final String localName = (String) m.get("localName");
final boolean isDir = (Boolean) m.get("isDir");
final boolean isSymlink = (Boolean) m.get("isSymlink");
final byte[] symlink = isSymlink?
DFSUtil.string2Bytes((String)m.get("symlink")): null;
final long len = (Long) m.get("len");
final String owner = (String) m.get("owner");
final String group = (String) m.get("group");
final FsPermission permission = toFsPermission((String) m.get("permission"));
final long aTime = (Long) m.get("accessTime");
final long mTime = (Long) m.get("modificationTime");
final long blockSize = (Long) m.get("blockSize");
final short replication = (short) (long) (Long) m.get("replication");
return new HdfsFileStatus(len, isDir, replication, blockSize, mTime, aTime,
permission, owner, group,
symlink, DFSUtil.string2Bytes(localName));
}
}

View File

@ -0,0 +1,345 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web;
import java.io.BufferedOutputStream;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.net.HttpURLConnection;
import java.net.URI;
import java.net.URISyntaxException;
import java.net.URL;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HftpFileSystem;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
import org.apache.hadoop.hdfs.web.resources.DeleteOpParam;
import org.apache.hadoop.hdfs.web.resources.DstPathParam;
import org.apache.hadoop.hdfs.web.resources.GetOpParam;
import org.apache.hadoop.hdfs.web.resources.GroupParam;
import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
import org.apache.hadoop.hdfs.web.resources.ModificationTimeParam;
import org.apache.hadoop.hdfs.web.resources.OverwriteParam;
import org.apache.hadoop.hdfs.web.resources.OwnerParam;
import org.apache.hadoop.hdfs.web.resources.Param;
import org.apache.hadoop.hdfs.web.resources.PermissionParam;
import org.apache.hadoop.hdfs.web.resources.PostOpParam;
import org.apache.hadoop.hdfs.web.resources.PutOpParam;
import org.apache.hadoop.hdfs.web.resources.RecursiveParam;
import org.apache.hadoop.hdfs.web.resources.RenameOptionSetParam;
import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Progressable;
import org.mortbay.util.ajax.JSON;
/** A FileSystem for HDFS over the web. */
public class WebHdfsFileSystem extends HftpFileSystem {
/** File System URI: {SCHEME}://namenode:port/path/to/file */
public static final String SCHEME = "webhdfs";
/** Http URI: http://namenode:port/{PATH_PREFIX}/path/to/file */
public static final String PATH_PREFIX = SCHEME;
private UserGroupInformation ugi;
protected Path workingDir;
@Override
public void initialize(URI uri, Configuration conf) throws IOException {
super.initialize(uri, conf);
setConf(conf);
ugi = UserGroupInformation.getCurrentUser();
this.workingDir = getHomeDirectory();
}
@Override
public URI getUri() {
try {
return new URI(SCHEME, null, nnAddr.getHostName(), nnAddr.getPort(),
null, null, null);
} catch (URISyntaxException e) {
return null;
}
}
@Override
public Path getHomeDirectory() {
return makeQualified(new Path("/user/" + ugi.getShortUserName()));
}
@Override
public synchronized Path getWorkingDirectory() {
return workingDir;
}
@Override
public synchronized void setWorkingDirectory(final Path dir) {
String result = makeAbsolute(dir).toUri().getPath();
if (!DFSUtil.isValidName(result)) {
throw new IllegalArgumentException("Invalid DFS directory name " +
result);
}
workingDir = makeAbsolute(dir);
}
private Path makeAbsolute(Path f) {
return f.isAbsolute()? f: new Path(workingDir, f);
}
@SuppressWarnings("unchecked")
private static Map<String, Object> jsonParse(final InputStream in
) throws IOException {
if (in == null) {
throw new IOException("The input stream is null.");
}
return (Map<String, Object>)JSON.parse(new InputStreamReader(in));
}
private static void validateResponse(final HttpOpParam.Op op,
final HttpURLConnection conn) throws IOException {
final int code = conn.getResponseCode();
if (code != op.getExpectedHttpResponseCode()) {
final Map<String, Object> m;
try {
m = jsonParse(conn.getErrorStream());
} catch(IOException e) {
throw new IOException("Unexpected HTTP response: code = " + code + " != "
+ op.getExpectedHttpResponseCode() + ", " + op.toQueryString()
+ ", message=" + conn.getResponseMessage(), e);
}
final RemoteException re = JsonUtil.toRemoteException(m);
throw re.unwrapRemoteException(AccessControlException.class,
DSQuotaExceededException.class,
FileAlreadyExistsException.class,
FileNotFoundException.class,
ParentNotDirectoryException.class,
SafeModeException.class,
NSQuotaExceededException.class,
UnresolvedPathException.class);
}
}
@Override
protected HttpURLConnection openConnection(String path, String query)
throws IOException {
query = addDelegationTokenParam(query);
final URL url = getNamenodeURL(path, query);
return (HttpURLConnection)url.openConnection();
}
private HttpURLConnection httpConnect(final HttpOpParam.Op op, final Path fspath,
final Param<?,?>... parameters) throws IOException {
//initialize URI path and query
final String uripath = "/" + PATH_PREFIX + makeQualified(fspath).toUri().getPath();
final String query = op.toQueryString() + Param.toSortedString("&", parameters);
//connect and get response
final HttpURLConnection conn = openConnection(uripath, query);
try {
conn.setRequestMethod(op.getType().toString());
conn.setDoOutput(op.getDoOutput());
if (op.getDoOutput()) {
conn.setRequestProperty("Expect", "100-Continue");
conn.setInstanceFollowRedirects(true);
}
conn.connect();
return conn;
} catch(IOException e) {
conn.disconnect();
throw e;
}
}
private Map<String, Object> run(final HttpOpParam.Op op, final Path fspath,
final Param<?,?>... parameters) throws IOException {
final HttpURLConnection conn = httpConnect(op, fspath, parameters);
validateResponse(op, conn);
try {
return jsonParse(conn.getInputStream());
} finally {
conn.disconnect();
}
}
private FsPermission applyUMask(FsPermission permission) {
if (permission == null) {
permission = FsPermission.getDefault();
}
return permission.applyUMask(FsPermission.getUMask(getConf()));
}
private HdfsFileStatus getHdfsFileStatus(Path f) throws IOException {
final HttpOpParam.Op op = GetOpParam.Op.GETFILESTATUS;
final Map<String, Object> json = run(op, f);
final HdfsFileStatus status = JsonUtil.toFileStatus(json);
if (status == null) {
throw new FileNotFoundException("File does not exist: " + f);
}
return status;
}
@Override
public FileStatus getFileStatus(Path f) throws IOException {
statistics.incrementReadOps(1);
return makeQualified(getHdfsFileStatus(f), f);
}
private FileStatus makeQualified(HdfsFileStatus f, Path parent) {
return new FileStatus(f.getLen(), f.isDir(), f.getReplication(),
f.getBlockSize(), f.getModificationTime(),
f.getAccessTime(),
f.getPermission(), f.getOwner(), f.getGroup(),
f.getFullPath(parent).makeQualified(getUri(), getWorkingDirectory()));
}
@Override
public boolean mkdirs(Path f, FsPermission permission) throws IOException {
statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PutOpParam.Op.MKDIRS;
final Map<String, Object> json = run(op, f,
new PermissionParam(applyUMask(permission)));
return (Boolean)json.get(op.toString());
}
@Override
public boolean rename(final Path src, final Path dst) throws IOException {
statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PutOpParam.Op.RENAME;
final Map<String, Object> json = run(op, src,
new DstPathParam(makeQualified(dst).toUri().getPath()));
return (Boolean)json.get(op.toString());
}
@SuppressWarnings("deprecation")
@Override
public void rename(final Path src, final Path dst,
final Options.Rename... options) throws IOException {
statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PutOpParam.Op.RENAME;
run(op, src, new DstPathParam(makeQualified(dst).toUri().getPath()),
new RenameOptionSetParam(options));
}
@Override
public void setOwner(final Path p, final String owner, final String group
) throws IOException {
if (owner == null && group == null) {
throw new IOException("owner == null && group == null");
}
statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PutOpParam.Op.SETOWNER;
run(op, p, new OwnerParam(owner), new GroupParam(group));
}
@Override
public void setPermission(final Path p, final FsPermission permission
) throws IOException {
statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PutOpParam.Op.SETPERMISSION;
run(op, p, new PermissionParam(permission));
}
@Override
public boolean setReplication(final Path p, final short replication
) throws IOException {
statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PutOpParam.Op.SETREPLICATION;
final Map<String, Object> json = run(op, p,
new ReplicationParam(replication));
return (Boolean)json.get(op.toString());
}
@Override
public void setTimes(final Path p, final long mtime, final long atime
) throws IOException {
statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PutOpParam.Op.SETTIMES;
run(op, p, new ModificationTimeParam(mtime), new AccessTimeParam(atime));
}
private FSDataOutputStream write(final HttpOpParam.Op op,
final HttpURLConnection conn, final int bufferSize) throws IOException {
return new FSDataOutputStream(new BufferedOutputStream(
conn.getOutputStream(), bufferSize), statistics) {
@Override
public void close() throws IOException {
try {
super.close();
} finally {
validateResponse(op, conn);
}
}
};
}
@Override
public FSDataOutputStream create(final Path f, final FsPermission permission,
final boolean overwrite, final int bufferSize, final short replication,
final long blockSize, final Progressable progress) throws IOException {
statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PutOpParam.Op.CREATE;
final HttpURLConnection conn = httpConnect(op, f,
new PermissionParam(applyUMask(permission)),
new OverwriteParam(overwrite),
new BufferSizeParam(bufferSize),
new ReplicationParam(replication),
new BlockSizeParam(blockSize));
return write(op, conn, bufferSize);
}
@Override
public FSDataOutputStream append(final Path f, final int bufferSize,
final Progressable progress) throws IOException {
statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PostOpParam.Op.APPEND;
final HttpURLConnection conn = httpConnect(op, f,
new BufferSizeParam(bufferSize));
return write(op, conn, bufferSize);
}
@Override
public boolean delete(Path f, boolean recursive) throws IOException {
final HttpOpParam.Op op = DeleteOpParam.Op.DELETE;
final Map<String, Object> json = run(op, f, new RecursiveParam(recursive));
return (Boolean)json.get(op.toString());
}
}

View File

@ -0,0 +1,49 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** Access time parameter. */
public class AccessTimeParam extends LongParam {
/** Parameter name. */
public static final String NAME = "accessTime";
/** Default parameter value. */
public static final String DEFAULT = "-1";
private static final Domain DOMAIN = new Domain(NAME);
/**
* Constructor.
* @param value the parameter value.
*/
public AccessTimeParam(final Long value) {
super(DOMAIN, value);
}
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public AccessTimeParam(final String str) {
this(DOMAIN.parse(str));
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,49 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** Block size parameter. */
public class BlockSizeParam extends LongParam {
/** Parameter name. */
public static final String NAME = "blockSize";
/** Default parameter value. */
public static final String DEFAULT = NULL;
private static final Domain DOMAIN = new Domain(NAME);
/**
* Constructor.
* @param value the parameter value.
*/
public BlockSizeParam(final Long value) {
super(DOMAIN, value);
}
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public BlockSizeParam(final String str) {
this(DOMAIN.parse(str));
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,51 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** Boolean parameter. */
abstract class BooleanParam extends Param<Boolean, BooleanParam.Domain> {
static final String TRUE = "true";
static final String FALSE = "false";
BooleanParam(final Domain domain, final Boolean value) {
super(domain, value);
}
/** The domain of the parameter. */
static final class Domain extends Param.Domain<Boolean> {
Domain(final String paramName) {
super(paramName);
}
@Override
public String getDomain() {
return "<" + NULL + " | boolean>";
}
@Override
Boolean parse(final String str) {
if (TRUE.equalsIgnoreCase(str)) {
return true;
} else if (FALSE.equalsIgnoreCase(str)) {
return false;
}
throw new IllegalArgumentException("Failed to parse \"" + str
+ "\" to Boolean.");
}
}
}

View File

@ -0,0 +1,49 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** Buffer size parameter. */
public class BufferSizeParam extends IntegerParam {
/** Parameter name. */
public static final String NAME = "bufferSize";
/** Default parameter value. */
public static final String DEFAULT = NULL;
private static final Domain DOMAIN = new Domain(NAME);
/**
* Constructor.
* @param value the parameter value.
*/
public BufferSizeParam(final Integer value) {
super(DOMAIN, value);
}
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public BufferSizeParam(final String str) {
this(DOMAIN.parse(str));
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,74 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
import java.net.HttpURLConnection;
/** Http DELETE operation parameter. */
public class DeleteOpParam extends HttpOpParam<DeleteOpParam.Op> {
/** Parameter name. */
public static final String NAME = "deleteOp";
/** Delete operations. */
public static enum Op implements HttpOpParam.Op {
DELETE(HttpURLConnection.HTTP_OK),
NULL(HttpURLConnection.HTTP_NOT_IMPLEMENTED);
final int expectedHttpResponseCode;
Op(final int expectedHttpResponseCode) {
this.expectedHttpResponseCode = expectedHttpResponseCode;
}
@Override
public HttpOpParam.Type getType() {
return HttpOpParam.Type.DELETE;
}
@Override
public boolean getDoOutput() {
return false;
}
@Override
public int getExpectedHttpResponseCode() {
return expectedHttpResponseCode;
}
@Override
public String toQueryString() {
return NAME + "=" + this;
}
}
private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public DeleteOpParam(final String str) {
super(DOMAIN, DOMAIN.parse(str));
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,43 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
import org.apache.hadoop.fs.Path;
/** Destination path parameter. */
public class DstPathParam extends StringParam {
/** Parameter name. */
public static final String NAME = "dstPath";
/** Default parameter value. */
public static final String DEFAULT = "";
private static final Domain DOMAIN = new Domain(NAME, null);
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public DstPathParam(final String str) {
super(DOMAIN, str == null || str.equals(DEFAULT)? null: new Path(str).toUri().getPath());
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,46 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
import java.util.Arrays;
abstract class EnumParam<E extends Enum<E>> extends Param<E, EnumParam.Domain<E>> {
EnumParam(final Domain<E> domain, final E value) {
super(domain, value);
}
/** The domain of the parameter. */
static final class Domain<E extends Enum<E>> extends Param.Domain<E> {
private final Class<E> enumClass;
Domain(String name, final Class<E> enumClass) {
super(name);
this.enumClass = enumClass;
}
@Override
public final String getDomain() {
return Arrays.asList(enumClass.getEnumConstants()).toString();
}
@Override
final E parse(final String str) {
return Enum.valueOf(enumClass, str.toUpperCase());
}
}
}

View File

@ -0,0 +1,86 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
import java.util.Arrays;
import java.util.EnumSet;
import java.util.Iterator;
abstract class EnumSetParam<E extends Enum<E>> extends Param<EnumSet<E>, EnumSetParam.Domain<E>> {
/** Convert an EnumSet to a string of comma separated values. */
static <E extends Enum<E>> String toString(EnumSet<E> set) {
if (set == null || set.isEmpty()) {
return "";
} else {
final StringBuilder b = new StringBuilder();
final Iterator<E> i = set.iterator();
b.append(i.next());
for(; i.hasNext(); ) {
b.append(',').append(i.next());
}
return b.toString();
}
}
static <E extends Enum<E>> EnumSet<E> toEnumSet(final Class<E> clazz,
final E... values) {
final EnumSet<E> set = EnumSet.noneOf(clazz);
set.addAll(Arrays.asList(values));
return set;
}
EnumSetParam(final Domain<E> domain, final EnumSet<E> value) {
super(domain, value);
}
@Override
public String toString() {
return getName() + "=" + toString(value);
}
/** The domain of the parameter. */
static final class Domain<E extends Enum<E>> extends Param.Domain<EnumSet<E>> {
private final Class<E> enumClass;
Domain(String name, final Class<E> enumClass) {
super(name);
this.enumClass = enumClass;
}
@Override
public final String getDomain() {
return Arrays.asList(enumClass.getEnumConstants()).toString();
}
/** The string contains a comma separated values. */
@Override
final EnumSet<E> parse(final String str) {
final EnumSet<E> set = EnumSet.noneOf(enumClass);
if (!str.isEmpty()) {
for(int i, j = 0; j >= 0; ) {
i = j;
j = str.indexOf(',', i+1);
final String sub = j >= 0? str.substring(i, j): str.substring(i);
set.add(Enum.valueOf(enumClass, sub.trim().toUpperCase()));
}
}
return set;
}
}
}

View File

@ -0,0 +1,59 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
import java.io.FileNotFoundException;
import java.io.IOException;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import javax.ws.rs.ext.ExceptionMapper;
import javax.ws.rs.ext.Provider;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.web.JsonUtil;
/** Handle exceptions. */
@Provider
public class ExceptionHandler implements ExceptionMapper<Exception> {
public static final Log LOG = LogFactory.getLog(ExceptionHandler.class);
@Override
public Response toResponse(final Exception e) {
if (LOG.isTraceEnabled()) {
LOG.trace("GOT EXCEPITION", e);
}
final Response.Status s;
if (e instanceof SecurityException) {
s = Response.Status.UNAUTHORIZED;
} else if (e instanceof FileNotFoundException) {
s = Response.Status.NOT_FOUND;
} else if (e instanceof IOException) {
s = Response.Status.FORBIDDEN;
} else if (e instanceof UnsupportedOperationException) {
s = Response.Status.BAD_REQUEST;
} else {
s = Response.Status.INTERNAL_SERVER_ERROR;
}
final String js = JsonUtil.toJsonString(e);
return Response.status(s).type(MediaType.APPLICATION_JSON).entity(js).build();
}
}

View File

@ -0,0 +1,73 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
import java.net.HttpURLConnection;
/** Http GET operation parameter. */
public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
/** Parameter name. */
public static final String NAME = "getOp";
/** Get operations. */
public static enum Op implements HttpOpParam.Op {
GETFILESTATUS(HttpURLConnection.HTTP_OK),
NULL(HttpURLConnection.HTTP_NOT_IMPLEMENTED);
final int expectedHttpResponseCode;
Op(final int expectedHttpResponseCode) {
this.expectedHttpResponseCode = expectedHttpResponseCode;
}
@Override
public HttpOpParam.Type getType() {
return HttpOpParam.Type.GET;
}
@Override
public boolean getDoOutput() {
return false;
}
@Override
public int getExpectedHttpResponseCode() {
return expectedHttpResponseCode;
}
@Override
public String toQueryString() {
return NAME + "=" + this;
}
}
private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public GetOpParam(final String str) {
super(DOMAIN, DOMAIN.parse(str));
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,41 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** Group parameter. */
public class GroupParam extends StringParam {
/** Parameter name. */
public static final String NAME = "group";
/** Default parameter value. */
public static final String DEFAULT = "";
private static final Domain DOMAIN = new Domain(NAME, null);
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public GroupParam(final String str) {
super(DOMAIN, str == null || str.equals(DEFAULT)? null: str);
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,52 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** Http operation parameter. */
public abstract class HttpOpParam<E extends Enum<E> & HttpOpParam.Op> extends EnumParam<E> {
/** Default parameter value. */
public static final String DEFAULT = NULL;
/** Http operation types */
public static enum Type {
GET, PUT, POST, DELETE;
}
/** Http operation interface. */
public static interface Op {
/** @return the Http operation type. */
public Type getType();
/** @return true if the operation has output. */
public boolean getDoOutput();
/** @return true if the operation has output. */
public int getExpectedHttpResponseCode();
/** @return a URI query string. */
public String toQueryString();
}
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
HttpOpParam(final Domain<E> domain, final E value) {
super(domain, value);
}
}

View File

@ -0,0 +1,60 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** Integer parameter. */
abstract class IntegerParam extends Param<Integer, IntegerParam.Domain> {
IntegerParam(final Domain domain, final Integer value) {
super(domain, value);
}
@Override
public String toString() {
return getName() + "=" + domain.toString(getValue());
}
/** The domain of the parameter. */
static final class Domain extends Param.Domain<Integer> {
/** The radix of the number. */
final int radix;
Domain(final String paramName) {
this(paramName, 10);
}
Domain(final String paramName, final int radix) {
super(paramName);
this.radix = radix;
}
@Override
public String getDomain() {
return "<" + NULL + " | int in radix " + radix + ">";
}
@Override
Integer parse(final String str) {
return NULL.equals(str)? null: Integer.parseInt(str, radix);
}
/** Convert an Integer to a String. */
String toString(final Integer n) {
return n == null? NULL: Integer.toString(n, radix);
}
}
}

View File

@ -0,0 +1,60 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** Long parameter. */
abstract class LongParam extends Param<Long, LongParam.Domain> {
LongParam(final Domain domain, final Long value) {
super(domain, value);
}
@Override
public String toString() {
return getName() + "=" + domain.toString(getValue());
}
/** The domain of the parameter. */
static final class Domain extends Param.Domain<Long> {
/** The radix of the number. */
final int radix;
Domain(final String paramName) {
this(paramName, 10);
}
Domain(final String paramName, final int radix) {
super(paramName);
this.radix = radix;
}
@Override
public String getDomain() {
return "<" + NULL + " | short in radix " + radix + ">";
}
@Override
Long parse(final String str) {
return NULL.equals(str)? null: Long.parseLong(str, radix);
}
/** Convert a Short to a String. */
String toString(final Long n) {
return n == null? NULL: Long.toString(n, radix);
}
}
}

View File

@ -0,0 +1,49 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** Modification time parameter. */
public class ModificationTimeParam extends LongParam {
/** Parameter name. */
public static final String NAME = "modificationTime";
/** Default parameter value. */
public static final String DEFAULT = "-1";
private static final Domain DOMAIN = new Domain(NAME);
/**
* Constructor.
* @param value the parameter value.
*/
public ModificationTimeParam(final Long value) {
super(DOMAIN, value);
}
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public ModificationTimeParam(final String str) {
this(DOMAIN.parse(str));
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,49 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** Recursive parameter. */
public class OverwriteParam extends BooleanParam {
/** Parameter name. */
public static final String NAME = "overwrite";
/** Default parameter value. */
public static final String DEFAULT = FALSE;
private static final Domain DOMAIN = new Domain(NAME);
/**
* Constructor.
* @param value the parameter value.
*/
public OverwriteParam(final Boolean value) {
super(DOMAIN, value);
}
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public OverwriteParam(final String str) {
this(DOMAIN.parse(str));
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,41 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** Owner parameter. */
public class OwnerParam extends StringParam {
/** Parameter name. */
public static final String NAME = "owner";
/** Default parameter value. */
public static final String DEFAULT = "";
private static final Domain DOMAIN = new Domain(NAME, null);
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public OwnerParam(final String str) {
super(DOMAIN, str == null || str.equals(DEFAULT)? null: str);
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,104 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
import java.util.Arrays;
import java.util.Comparator;
/** Base class of parameters. */
public abstract class Param<T, D extends Param.Domain<T>> {
static final String NULL = "null";
static final Comparator<Param<?,?>> NAME_CMP = new Comparator<Param<?,?>>() {
@Override
public int compare(Param<?, ?> left, Param<?, ?> right) {
return left.getName().compareTo(right.getName());
}
};
/** Convert the parameters to a sorted String. */
public static String toSortedString(final String separator,
final Param<?, ?>... parameters) {
Arrays.sort(parameters, NAME_CMP);
final StringBuilder b = new StringBuilder();
for(Param<?, ?> p : parameters) {
if (p.getValue() != null) {
b.append(separator).append(p);
}
}
return b.toString();
}
/** The domain of the parameter. */
final D domain;
/** The actual parameter value. */
final T value;
Param(final D domain, final T value) {
this.domain = domain;
this.value = value;
}
/** @return the parameter value. */
public final T getValue() {
return value;
}
/** @return the parameter name. */
public abstract String getName();
@Override
public String toString() {
return getName() + "=" + value;
}
/** Base class of parameter domains. */
static abstract class Domain<T> {
/** Parameter name. */
final String paramName;
Domain(final String paramName) {
this.paramName = paramName;
}
/** @return the parameter name. */
public final String getParamName() {
return paramName;
}
/** @return a string description of the domain of the parameter. */
public abstract String getDomain();
/** @return the parameter value represented by the string. */
abstract T parse(String str);
/** Parse the given string.
* @return the parameter value represented by the string.
*/
public final T parse(final String varName, final String str) {
try {
return str != null && str.trim().length() > 0 ? parse(str) : null;
} catch(Exception e) {
throw new IllegalArgumentException("Failed to parse \"" + str
+ "\" for the parameter " + varName
+ ". The value must be in the domain " + getDomain(), e);
}
}
}
}

View File

@ -0,0 +1,57 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
import org.apache.hadoop.fs.permission.FsPermission;
/** Permission parameter, use a Short to represent a FsPermission. */
public class PermissionParam extends ShortParam {
/** Parameter name. */
public static final String NAME = "permission";
/** Default parameter value. */
public static final String DEFAULT = NULL;
private static final Domain DOMAIN = new Domain(NAME, 8);
/**
* Constructor.
* @param value the parameter value.
*/
public PermissionParam(final FsPermission value) {
super(DOMAIN, value == null? null: value.toShort());
}
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public PermissionParam(final String str) {
super(DOMAIN, DOMAIN.parse(str));
}
@Override
public String getName() {
return NAME;
}
/** @return the represented FsPermission. */
public FsPermission getFsPermission() {
final Short mode = getValue();
return mode == null? FsPermission.getDefault(): new FsPermission(mode);
}
}

View File

@ -0,0 +1,74 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
import java.net.HttpURLConnection;
/** Http POST operation parameter. */
public class PostOpParam extends HttpOpParam<PostOpParam.Op> {
/** Parameter name. */
public static final String NAME = "postOp";
/** Post operations. */
public static enum Op implements HttpOpParam.Op {
APPEND(HttpURLConnection.HTTP_OK),
NULL(HttpURLConnection.HTTP_NOT_IMPLEMENTED);
final int expectedHttpResponseCode;
Op(final int expectedHttpResponseCode) {
this.expectedHttpResponseCode = expectedHttpResponseCode;
}
@Override
public Type getType() {
return Type.POST;
}
@Override
public boolean getDoOutput() {
return true;
}
@Override
public int getExpectedHttpResponseCode() {
return expectedHttpResponseCode;
}
/** @return a URI query string. */
public String toQueryString() {
return NAME + "=" + this;
}
}
private static final Domain<Op> DOMAIN = new Domain<PostOpParam.Op>(NAME, Op.class);
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public PostOpParam(final String str) {
super(DOMAIN, DOMAIN.parse(str));
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,84 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
import java.net.HttpURLConnection;
/** Http POST operation parameter. */
public class PutOpParam extends HttpOpParam<PutOpParam.Op> {
/** Parameter name. */
public static final String NAME = "putOp";
/** Put operations. */
public static enum Op implements HttpOpParam.Op {
CREATE(true, HttpURLConnection.HTTP_CREATED),
MKDIRS(false, HttpURLConnection.HTTP_OK),
RENAME(false, HttpURLConnection.HTTP_OK),
SETREPLICATION(false, HttpURLConnection.HTTP_OK),
SETOWNER(false, HttpURLConnection.HTTP_OK),
SETPERMISSION(false, HttpURLConnection.HTTP_OK),
SETTIMES(false, HttpURLConnection.HTTP_OK),
NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED);
final boolean doOutput;
final int expectedHttpResponseCode;
Op(final boolean doOutput, final int expectedHttpResponseCode) {
this.doOutput = doOutput;
this.expectedHttpResponseCode = expectedHttpResponseCode;
}
@Override
public HttpOpParam.Type getType() {
return HttpOpParam.Type.PUT;
}
@Override
public boolean getDoOutput() {
return doOutput;
}
@Override
public int getExpectedHttpResponseCode() {
return expectedHttpResponseCode;
}
@Override
public String toQueryString() {
return NAME + "=" + this;
}
}
private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public PutOpParam(final String str) {
super(DOMAIN, DOMAIN.parse(str));
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,49 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** Recursive parameter. */
public class RecursiveParam extends BooleanParam {
/** Parameter name. */
public static final String NAME = "recursive";
/** Default parameter value. */
public static final String DEFAULT = FALSE;
private static final Domain DOMAIN = new Domain(NAME);
/**
* Constructor.
* @param value the parameter value.
*/
public RecursiveParam(final Boolean value) {
super(DOMAIN, value);
}
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public RecursiveParam(final String str) {
this(DOMAIN.parse(str));
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,52 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
import org.apache.hadoop.fs.Options;
/** Rename option set parameter. */
public class RenameOptionSetParam extends EnumSetParam<Options.Rename> {
/** Parameter name. */
public static final String NAME = "renameOptions";
/** Default parameter value. */
public static final String DEFAULT = "";
private static final Domain<Options.Rename> DOMAIN = new Domain<Options.Rename>(
NAME, Options.Rename.class);
/**
* Constructor.
* @param options rename options.
*/
public RenameOptionSetParam(final Options.Rename... options) {
super(DOMAIN, toEnumSet(Options.Rename.class, options));
}
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public RenameOptionSetParam(final String str) {
super(DOMAIN, DOMAIN.parse(str));
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,49 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** Replication parameter. */
public class ReplicationParam extends ShortParam {
/** Parameter name. */
public static final String NAME = "replication";
/** Default parameter value. */
public static final String DEFAULT = NULL;
private static final Domain DOMAIN = new Domain(NAME);
/**
* Constructor.
* @param value the parameter value.
*/
public ReplicationParam(final Short value) {
super(DOMAIN, value);
}
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public ReplicationParam(final String str) {
this(DOMAIN.parse(str));
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,60 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** Short parameter. */
abstract class ShortParam extends Param<Short, ShortParam.Domain> {
ShortParam(final Domain domain, final Short value) {
super(domain, value);
}
@Override
public String toString() {
return getName() + "=" + domain.toString(getValue());
}
/** The domain of the parameter. */
static final class Domain extends Param.Domain<Short> {
/** The radix of the number. */
final int radix;
Domain(final String paramName) {
this(paramName, 10);
}
Domain(final String paramName, final int radix) {
super(paramName);
this.radix = radix;
}
@Override
public String getDomain() {
return "<" + NULL + " | short in radix " + radix + ">";
}
@Override
Short parse(final String str) {
return NULL.equals(str)? null: Short.parseShort(str, radix);
}
/** Convert a Short to a String. */
String toString(final Short n) {
return n == null? NULL: Integer.toString(n, radix);
}
}
}

View File

@ -0,0 +1,54 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
import java.util.regex.Pattern;
/** String parameter. */
abstract class StringParam extends Param<String, StringParam.Domain> {
StringParam(final Domain domain, String str) {
super(domain, domain.parse(str));
}
/** The domain of the parameter. */
static final class Domain extends Param.Domain<String> {
/** The pattern defining the domain; null . */
private final Pattern pattern;
Domain(final String paramName, final Pattern pattern) {
super(paramName);
this.pattern = pattern;
}
@Override
public final String getDomain() {
return pattern == null ? "<String>" : pattern.pattern();
}
@Override
final String parse(final String str) {
if (pattern != null) {
if (!pattern.matcher(str).matches()) {
throw new IllegalArgumentException("Invalid value: \"" + str
+ "\" does not belong to the domain " + getDomain());
}
}
return str;
}
}
}

View File

@ -0,0 +1,45 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** The FileSystem path parameter. */
public class UriFsPathParam extends StringParam {
/** Parameter name. */
public static final String NAME = "path";
private static final Domain DOMAIN = new Domain(NAME, null);
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public UriFsPathParam(String str) {
super(DOMAIN, str);
}
@Override
public String getName() {
return NAME;
}
/** @return the absolute path. */
public final String getAbsolutePath() {
final String path = getValue();
return path == null? null: "/" + path;
}
}

View File

@ -0,0 +1,41 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
/** User parameter. */
public class UserParam extends StringParam {
/** Parameter name. */
public static final String NAME = "user.name";
/** Default parameter value. */
public static final String DEFAULT = "";
private static final Domain DOMAIN = new Domain(NAME, null);
/**
* Constructor.
* @param str a string representation of the parameter value.
*/
public UserParam(final String str) {
super(DOMAIN, str == null || str.equals(DEFAULT)? null: str);
}
@Override
public String getName() {
return NAME;
}
}

View File

@ -0,0 +1,73 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web.resources;
import java.lang.reflect.Type;
import java.security.Principal;
import javax.ws.rs.core.Context;
import javax.ws.rs.ext.Provider;
import com.sun.jersey.api.core.HttpContext;
import com.sun.jersey.core.spi.component.ComponentContext;
import com.sun.jersey.core.spi.component.ComponentScope;
import com.sun.jersey.server.impl.inject.AbstractHttpContextInjectable;
import com.sun.jersey.spi.inject.Injectable;
import com.sun.jersey.spi.inject.InjectableProvider;
@Provider
public class UserProvider extends AbstractHttpContextInjectable<Principal>
implements InjectableProvider<Context, Type> {
@Override
public Principal getValue(final HttpContext context) {
//get principal from the request
final Principal principal = context.getRequest().getUserPrincipal();
if (principal != null) {
return principal;
}
//get username from the parameter
final String username = context.getRequest().getQueryParameters().getFirst(
UserParam.NAME);
if (username != null) {
final UserParam userparam = new UserParam(username);
return new Principal() {
@Override
public String getName() {
return userparam.getValue();
}
};
}
//user not found
return null;
}
@Override
public ComponentScope getScope() {
return ComponentScope.PerRequest;
}
@Override
public Injectable<Principal> getInjectable(
final ComponentContext componentContext, final Context context,
final Type type) {
return type.equals(Principal.class)? this : null;
}
}

View File

@ -63,6 +63,7 @@
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.hdfs.tools.DFSAdmin;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.net.DNSToSwitchMapping;
import org.apache.hadoop.net.NetUtils;
@ -815,6 +816,8 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes,
long[] simulatedCapacities,
boolean setupHostsFile,
boolean checkDataNodeAddrConfig) throws IOException {
conf.set(DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY, "127.0.0.1");
int curDatanodesNum = dataNodes.size();
// for mincluster's the default initialDelay for BRs is 0
if (conf.get(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY) == null) {
@ -1458,6 +1461,18 @@ public HftpFileSystem getHftpFileSystem(int nnIndex) throws IOException {
}
}
/**
* @return a {@link WebHdfsFileSystem} object.
*/
public WebHdfsFileSystem getWebHdfsFileSystem() throws IOException {
final String str = WebHdfsFileSystem.SCHEME + "://" + conf.get("dfs.http.address");
try {
return (WebHdfsFileSystem)FileSystem.get(new URI(str), conf);
} catch (URISyntaxException e) {
throw new IOException(e);
}
}
/**
* @return a {@link HftpFileSystem} object as specified user.
*/

View File

@ -0,0 +1,96 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web;
import static org.apache.hadoop.fs.FileSystemTestHelper.exists;
import static org.apache.hadoop.fs.FileSystemTestHelper.getDefaultBlockSize;
import static org.apache.hadoop.fs.FileSystemTestHelper.getTestRootPath;
import java.io.IOException;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FSMainOperationsBaseTest;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.web.resources.ExceptionHandler;
import org.apache.log4j.Level;
import org.junit.Assert;
import org.junit.Test;
public class TestFSMainOperationsWebHdfs extends FSMainOperationsBaseTest {
{
((Log4JLogger)ExceptionHandler.LOG).getLogger().setLevel(Level.ALL);
}
private static final MiniDFSCluster cluster;
private static final Path defaultWorkingDirectory;
static {
Configuration conf = new Configuration();
try {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
fSys = cluster.getWebHdfsFileSystem();
defaultWorkingDirectory = fSys.getWorkingDirectory();
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
protected Path getDefaultWorkingDirectory() {
return defaultWorkingDirectory;
}
/** Override the following method without using position read. */
@Override
protected void writeReadAndDelete(int len) throws IOException {
Path path = getTestRootPath(fSys, "test/hadoop/file");
fSys.mkdirs(path.getParent());
FSDataOutputStream out =
fSys.create(path, false, 4096, (short) 1, getDefaultBlockSize() );
out.write(data, 0, len);
out.close();
Assert.assertTrue("Exists", exists(fSys, path));
Assert.assertEquals("Length", len, fSys.getFileStatus(path).getLen());
FSDataInputStream in = fSys.open(path);
for (int i = 0; i < len; i++) {
final int b = in.read();
Assert.assertEquals("Position " + i, data[i], b);
}
in.close();
Assert.assertTrue("Deleted", fSys.delete(path, false));
Assert.assertFalse("No longer exists", exists(fSys, path));
}
//The following tests failed for HftpFileSystem,
//Disable it for WebHdfsFileSystem
@Test
public void testListStatusThrowsExceptionForNonExistentFile() {}
@Test
public void testListStatusThrowsExceptionForUnreadableDir() {}
@Test
public void testGlobStatusThrowsExceptionForNonExistentFile() {}
}

View File

@ -0,0 +1,55 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.web.JsonUtil;
import org.junit.Assert;
import org.junit.Test;
public class TestJsonUtil {
static FileStatus toFileStatus(HdfsFileStatus f, String parent) {
return new FileStatus(f.getLen(), f.isDir(), f.getReplication(),
f.getBlockSize(), f.getModificationTime(), f.getAccessTime(),
f.getPermission(), f.getOwner(), f.getGroup(),
new Path(f.getFullName(parent)));
}
@Test
public void testHdfsFileStatus() {
final long now = System.currentTimeMillis();
final String parent = "/dir";
final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L<<26,
now, now + 10, new FsPermission((short)0644), "user", "group",
DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"));
final FileStatus fstatus = toFileStatus(status, parent);
System.out.println("status = " + status);
System.out.println("fstatus = " + fstatus);
final String json = JsonUtil.toJsonString(status);
System.out.println("json = " + json.replace(",", ",\n "));
final HdfsFileStatus s2 = JsonUtil.toFileStatus(JsonUtil.parse(json));
final FileStatus fs2 = toFileStatus(s2, parent);
System.out.println("s2 = " + s2);
System.out.println("fs2 = " + fs2);
Assert.assertEquals(fstatus, fs2);
}
}

View File

@ -0,0 +1,86 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.web;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystemContractBaseTest;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.security.UserGroupInformation;
public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest {
private static final MiniDFSCluster cluster;
private String defaultWorkingDirectory;
static {
Configuration conf = new Configuration();
try {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
protected void setUp() throws Exception {
fs = cluster.getWebHdfsFileSystem();
defaultWorkingDirectory = "/user/"
+ UserGroupInformation.getCurrentUser().getShortUserName();
}
@Override
protected String getDefaultWorkingDirectory() {
return defaultWorkingDirectory;
}
/** Override the following method without using position read. */
@Override
protected void writeReadAndDelete(int len) throws IOException {
Path path = path("/test/hadoop/file");
fs.mkdirs(path.getParent());
FSDataOutputStream out = fs.create(path, false,
fs.getConf().getInt("io.file.buffer.size", 4096),
(short) 1, getBlockSize());
out.write(data, 0, len);
out.close();
assertTrue("Exists", fs.exists(path));
assertEquals("Length", len, fs.getFileStatus(path).getLen());
FSDataInputStream in = fs.open(path);
for (int i = 0; i < len; i++) {
final int b = in.read();
assertEquals("Position " + i, data[i], b);
}
in.close();
assertTrue("Deleted", fs.delete(path, false));
assertFalse("No longer exists", fs.exists(path));
}
//The following test failed for HftpFileSystem,
//Disable it for WebHdfsFileSystem
public void testListStatusThrowsExceptionForNonExistentFile() {}
}