svn merge -c 1183554 from trunk for HDFS-2439.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1189491 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
9c5bdd0e66
commit
01671cd290
|
@ -1,7 +1,5 @@
|
|||
Hadoop HDFS Change Log
|
||||
|
||||
MAPREDUCE-2764. Fix renewal of dfs delegation tokens. (Owen via jitendra)
|
||||
|
||||
Release 0.23.0 - Unreleased
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -1107,6 +1105,12 @@ Release 0.23.0 - Unreleased
|
|||
HDFS-2424. Added a root element "HdfsFileStatuses" for the response
|
||||
of webhdfs listStatus. (szetszwo)
|
||||
|
||||
MAPREDUCE-2764. Fix renewal of dfs delegation tokens. (Owen via jitendra)
|
||||
|
||||
HDFS-2439. Fix NullPointerException in webhdfs when opening a non-existing
|
||||
file or creating a file without specifying the replication parameter.
|
||||
(szetszwo)
|
||||
|
||||
BREAKDOWN OF HDFS-1073 SUBTASKS
|
||||
|
||||
HDFS-1521. Persist transaction ID on disk between NN restarts.
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
|
||||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.HttpURLConnection;
|
||||
|
@ -107,6 +108,8 @@ public class ByteRangeInputStream extends FSInputStream {
|
|||
HftpFileSystem.LOG.debug("filelength = " + filelength);
|
||||
}
|
||||
in = connection.getInputStream();
|
||||
} catch (FileNotFoundException fnfe) {
|
||||
throw fnfe;
|
||||
} catch (IOException ioe) {
|
||||
HftpFileSystem.throwIOExceptionFromConnection(connection, ioe);
|
||||
}
|
||||
|
|
|
@ -131,7 +131,7 @@ public class DatanodeWebHdfsMethods {
|
|||
fullpath, permission.getFsPermission(),
|
||||
overwrite.getValue() ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
|
||||
: EnumSet.of(CreateFlag.CREATE),
|
||||
replication.getValue(), blockSize.getValue(conf), null, b), null);
|
||||
replication.getValue(conf), blockSize.getValue(conf), null, b), null);
|
||||
try {
|
||||
IOUtils.copyBytes(in, out, b);
|
||||
} finally {
|
||||
|
|
|
@ -46,6 +46,7 @@ import javax.ws.rs.core.StreamingOutput;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.ContentSummary;
|
||||
import org.apache.hadoop.fs.Options;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
|
@ -119,6 +120,9 @@ public class NamenodeWebHdfsMethods {
|
|||
|| op == PostOpParam.Op.APPEND) {
|
||||
final NamenodeProtocols np = namenode.getRpcServer();
|
||||
final HdfsFileStatus status = np.getFileInfo(path);
|
||||
if (status == null) {
|
||||
throw new FileNotFoundException("File " + path + " not found.");
|
||||
}
|
||||
final long len = status.getLen();
|
||||
if (op == GetOpParam.Op.OPEN && (openOffset < 0L || openOffset >= len)) {
|
||||
throw new IOException("Offset=" + openOffset + " out of the range [0, "
|
||||
|
@ -238,6 +242,7 @@ public class NamenodeWebHdfsMethods {
|
|||
try {
|
||||
|
||||
final String fullpath = path.getAbsolutePath();
|
||||
final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
|
||||
final NameNode namenode = (NameNode)context.getAttribute("name.node");
|
||||
final NamenodeProtocols np = namenode.getRpcServer();
|
||||
|
||||
|
@ -259,7 +264,6 @@ public class NamenodeWebHdfsMethods {
|
|||
{
|
||||
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("boolean", b);
|
||||
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
|
||||
|
@ -271,7 +275,7 @@ public class NamenodeWebHdfsMethods {
|
|||
}
|
||||
case SETREPLICATION:
|
||||
{
|
||||
final boolean b = np.setReplication(fullpath, replication.getValue());
|
||||
final boolean b = np.setReplication(fullpath, replication.getValue(conf));
|
||||
final String js = JsonUtil.toJsonString("boolean", b);
|
||||
return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
|
||||
}
|
||||
|
|
|
@ -17,6 +17,11 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs.web.resources;
|
||||
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
/** Replication parameter. */
|
||||
public class ReplicationParam extends ShortParam {
|
||||
/** Parameter name. */
|
||||
|
@ -46,4 +51,10 @@ public class ReplicationParam extends ShortParam {
|
|||
public String getName() {
|
||||
return NAME;
|
||||
}
|
||||
|
||||
/** @return the value or, if it is null, return the default from conf. */
|
||||
public short getValue(final Configuration conf) {
|
||||
return getValue() != null? getValue()
|
||||
: (short)conf.getInt(DFS_REPLICATION_KEY, DFS_REPLICATION_DEFAULT);
|
||||
}
|
||||
}
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hdfs.web;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.net.HttpURLConnection;
|
||||
|
@ -28,6 +29,7 @@ import java.security.PrivilegedExceptionAction;
|
|||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.BlockLocation;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileSystemContractBaseTest;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -158,4 +160,16 @@ public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest {
|
|||
//check if the command successes.
|
||||
assertTrue(fs.getFileStatus(p).isDirectory());
|
||||
}
|
||||
|
||||
public void testOpenNonExistFile() throws IOException {
|
||||
final Path p = new Path("/test/testOpenNonExistFile");
|
||||
//open it as a file, should get FileNotFoundException
|
||||
try {
|
||||
final FSDataInputStream in = fs.open(p);
|
||||
in.read();
|
||||
fail();
|
||||
} catch(FileNotFoundException fnfe) {
|
||||
WebHdfsFileSystem.LOG.info("This is expected.", fnfe);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue