HDFS-5110 Change FSDataOutputStream to HdfsDataOutputStream for opened streams to fix type cast error. Contributed by Brandon Li

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1515624 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Brandon Li 2013-08-19 21:54:51 +00:00
parent be0317e021
commit c9b89de0ea
3 changed files with 11 additions and 8 deletions

View File

@ -32,7 +32,6 @@ import java.util.concurrent.locks.ReentrantLock;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream; import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag; import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
@ -70,7 +69,7 @@ class OpenFileCtx {
// The stream write-back status. True means one thread is doing write back. // The stream write-back status. True means one thread is doing write back.
private boolean asyncStatus; private boolean asyncStatus;
private final FSDataOutputStream fos; private final HdfsDataOutputStream fos;
private final Nfs3FileAttributes latestAttr; private final Nfs3FileAttributes latestAttr;
private long nextOffset; private long nextOffset;
@ -114,7 +113,7 @@ class OpenFileCtx {
return nonSequentialWriteInMemory; return nonSequentialWriteInMemory;
} }
OpenFileCtx(FSDataOutputStream fos, Nfs3FileAttributes latestAttr, OpenFileCtx(HdfsDataOutputStream fos, Nfs3FileAttributes latestAttr,
String dumpFilePath) { String dumpFilePath) {
this.fos = fos; this.fos = fos;
this.latestAttr = latestAttr; this.latestAttr = latestAttr;
@ -438,7 +437,7 @@ class OpenFileCtx {
FSDataInputStream fis = null; FSDataInputStream fis = null;
try { try {
// Sync file data and length to avoid partial read failure // Sync file data and length to avoid partial read failure
((HdfsDataOutputStream) fos).hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH)); fos.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
fis = new FSDataInputStream(dfsClient.open(path)); fis = new FSDataInputStream(dfsClient.open(path));
readCount = fis.read(offset, readbuffer, 0, count); readCount = fis.read(offset, readbuffer, 0, count);
@ -527,7 +526,7 @@ class OpenFileCtx {
int ret = COMMIT_WAIT; int ret = COMMIT_WAIT;
try { try {
// Sync file data and length // Sync file data and length
((HdfsDataOutputStream) fos).hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH)); fos.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
// Nothing to do for metadata since attr related change is pass-through // Nothing to do for metadata since attr related change is pass-through
ret = COMMIT_FINISHED; ret = COMMIT_FINISHED;
} catch (IOException e) { } catch (IOException e) {

View File

@ -29,7 +29,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.FileSystem.Statistics; import org.apache.hadoop.fs.FileSystem.Statistics;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
@ -629,7 +629,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
return new CREATE3Response(Nfs3Status.NFS3ERR_INVAL); return new CREATE3Response(Nfs3Status.NFS3ERR_INVAL);
} }
FSDataOutputStream fos = null; HdfsDataOutputStream fos = null;
String dirFileIdPath = Nfs3Utils.getFileIdPath(dirHandle); String dirFileIdPath = Nfs3Utils.getFileIdPath(dirHandle);
WccAttr preOpDirAttr = null; WccAttr preOpDirAttr = null;
Nfs3FileAttributes postOpObjAttr = null; Nfs3FileAttributes postOpObjAttr = null;
@ -652,7 +652,8 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
EnumSet<CreateFlag> flag = (createMode != Nfs3Constant.CREATE_EXCLUSIVE) ? EnumSet EnumSet<CreateFlag> flag = (createMode != Nfs3Constant.CREATE_EXCLUSIVE) ? EnumSet
.of(CreateFlag.CREATE, CreateFlag.OVERWRITE) : EnumSet .of(CreateFlag.CREATE, CreateFlag.OVERWRITE) : EnumSet
.of(CreateFlag.CREATE); .of(CreateFlag.CREATE);
fos = new FSDataOutputStream(dfsClient.create(fileIdPath, permission,
fos = new HdfsDataOutputStream(dfsClient.create(fileIdPath, permission,
flag, false, replication, blockSize, null, bufferSize, null), flag, false, replication, blockSize, null, bufferSize, null),
statistics); statistics);

View File

@ -296,6 +296,9 @@ Release 2.1.1-beta - UNRELEASED
HDFS-5107 Fix array copy error in Readdir and Readdirplus responses HDFS-5107 Fix array copy error in Readdir and Readdirplus responses
(brandonli) (brandonli)
HDFS-5110 Change FSDataOutputStream to HdfsDataOutputStream for opened
streams to fix type cast error. (brandonli)
IMPROVEMENTS IMPROVEMENTS
HDFS-4513. Clarify in the WebHDFS REST API that all JSON respsonses may HDFS-4513. Clarify in the WebHDFS REST API that all JSON respsonses may