You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by br...@apache.org on 2013/08/19 23:54:51 UTC

svn commit: r1515624 - in /hadoop/common/trunk/hadoop-hdfs-project: hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java hadoop-hdfs/CHANGES.txt

Author: brandonli
Date: Mon Aug 19 21:54:51 2013
New Revision: 1515624

URL: http://svn.apache.org/r1515624
Log:
HDFS-5110 Change FSDataOutputStream to HdfsDataOutputStream for opened streams to fix type cast error. Contributed by Brandon Li

Modified:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java?rev=1515624&r1=1515623&r2=1515624&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java Mon Aug 19 21:54:51 2013
@@ -32,7 +32,6 @@ import java.util.concurrent.locks.Reentr
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 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.
   private boolean asyncStatus;
 
-  private final FSDataOutputStream fos;
+  private final HdfsDataOutputStream fos;
   private final Nfs3FileAttributes latestAttr;
   private long nextOffset;
 
@@ -114,7 +113,7 @@ class OpenFileCtx {
     return nonSequentialWriteInMemory;
   }
   
-  OpenFileCtx(FSDataOutputStream fos, Nfs3FileAttributes latestAttr,
+  OpenFileCtx(HdfsDataOutputStream fos, Nfs3FileAttributes latestAttr,
       String dumpFilePath) {
     this.fos = fos;
     this.latestAttr = latestAttr;
@@ -438,7 +437,7 @@ class OpenFileCtx {
     FSDataInputStream fis = null;
     try {
       // 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));
       readCount = fis.read(offset, readbuffer, 0, count);
@@ -527,7 +526,7 @@ class OpenFileCtx {
     int ret = COMMIT_WAIT;
     try {
       // 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
       ret = COMMIT_FINISHED;
     } catch (IOException e) {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java?rev=1515624&r1=1515623&r2=1515624&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java Mon Aug 19 21:54:51 2013
@@ -29,7 +29,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
 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.FileSystem.Statistics;
 import org.apache.hadoop.fs.FileUtil;
@@ -629,7 +629,7 @@ public class RpcProgramNfs3 extends RpcP
       return new CREATE3Response(Nfs3Status.NFS3ERR_INVAL);
     }
 
-    FSDataOutputStream fos = null;
+    HdfsDataOutputStream fos = null;
     String dirFileIdPath = Nfs3Utils.getFileIdPath(dirHandle);
     WccAttr preOpDirAttr = null;
     Nfs3FileAttributes postOpObjAttr = null;
@@ -652,7 +652,8 @@ public class RpcProgramNfs3 extends RpcP
       EnumSet<CreateFlag> flag = (createMode != Nfs3Constant.CREATE_EXCLUSIVE) ? EnumSet
           .of(CreateFlag.CREATE, CreateFlag.OVERWRITE) : EnumSet
           .of(CreateFlag.CREATE);
-      fos = new FSDataOutputStream(dfsClient.create(fileIdPath, permission,
+      
+      fos = new HdfsDataOutputStream(dfsClient.create(fileIdPath, permission,
           flag, false, replication, blockSize, null, bufferSize, null),
           statistics);
       

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1515624&r1=1515623&r2=1515624&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Mon Aug 19 21:54:51 2013
@@ -296,6 +296,9 @@ Release 2.1.1-beta - UNRELEASED
     HDFS-5107 Fix array copy error in Readdir and Readdirplus responses
     (brandonli)
 
+    HDFS-5110 Change FSDataOutputStream to HdfsDataOutputStream for opened
+    streams to fix type cast error. (brandonli)
+
   IMPROVEMENTS
 
     HDFS-4513. Clarify in the WebHDFS REST API that all JSON respsonses may