浏览代码

HDFS-2333. Change DFSOutputStream back to package private, otherwise, there are two SC_START_IN_CTOR findbugs warnings. (szetszwo)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security@1171137 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 13 年之前
父节点
当前提交
ed1184ad08

+ 3 - 0
CHANGES.txt

@@ -157,6 +157,9 @@ Release 0.20.205.0 - unreleased
     HDFS-2331. Fix WebHdfsFileSystem compilation problems for a bug in JDK
     version < 1.6.0_26.  (Abhijit Suresh Shingate via szetszwo)
 
+    HDFS-2333. Change DFSOutputStream back to package private, otherwise,
+    there are two SC_START_IN_CTOR findbugs warnings.  (szetszwo)
+
   IMPROVEMENTS
 
     MAPREDUCE-2187. Reporter sends progress during sort/merge. (Anupam Seth via

+ 15 - 7
src/hdfs/org/apache/hadoop/hdfs/DFSClient.java

@@ -641,12 +641,20 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * 
    * @param src file name
    * @param buffersize buffer size
-   * @param progress for reporting write-progress
+   * @param progress for reporting write-progress; null is acceptable.
+   * @param statistics file system statistics; null is acceptable.
    * @return an output stream for writing into the file
-   * @throws IOException
-   * @see ClientProtocol#append(String, String)
+   * 
+   * @see ClientProtocol#append(String, String) 
    */
-  public DFSOutputStream append(String src, int buffersize, Progressable progress
+  public FSDataOutputStream append(final String src, final int buffersize,
+      final Progressable progress, final FileSystem.Statistics statistics
+      ) throws IOException {
+    final DFSOutputStream out = append(src, buffersize, progress);
+    return new FSDataOutputStream(out, statistics, out.getInitialLen());
+  }
+
+  private DFSOutputStream append(String src, int buffersize, Progressable progress
       ) throws IOException {
     checkOpen();
     HdfsFileStatus stat = null;
@@ -2415,7 +2423,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
    * datanode from the original pipeline. The DataStreamer now
    * starts sending packets from the dataQueue.
   ****************************************************************/
-  public class DFSOutputStream extends FSOutputSummer implements Syncable {
+  class DFSOutputStream extends FSOutputSummer implements Syncable {
     private Socket s;
     boolean closed = false;
   
@@ -3581,7 +3589,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
      * block is not yet allocated, then this API will return 0 because there are
      * no replicas in the pipeline.
      */
-    public int getNumCurrentReplicas() throws IOException {
+    int getNumCurrentReplicas() throws IOException {
       synchronized(dataQueue) {
         if (nodes == null) {
           return blockReplication;
@@ -3754,7 +3762,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     /**
      * Returns the size of a file as it was when this stream was opened
      */
-    public long getInitialLen() {
+    long getInitialLen() {
       return initialFileSize;
     }
   }

+ 1 - 3
src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -202,10 +202,8 @@ public class DistributedFileSystem extends FileSystem {
   /** This optional operation is not yet supported. */
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException {
-
     statistics.incrementWriteOps(1);
-    final DFSOutputStream op = dfs.append(getPathName(f), bufferSize, progress);
-    return new FSDataOutputStream(op, statistics, op.getInitialLen());
+    return dfs.append(getPathName(f), bufferSize, progress, statistics);
   }
 
   public FSDataOutputStream create(Path f, FsPermission permission,

+ 2 - 5
src/hdfs/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java

@@ -45,7 +45,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
-import org.apache.hadoop.hdfs.DFSClient.DFSOutputStream;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
@@ -149,10 +148,8 @@ public class DatanodeWebHdfsMethods {
     {
       final Configuration conf = new Configuration(datanode.getConf());
       final DFSClient dfsclient = new DFSClient(conf);
-      final DFSOutputStream dfsout = dfsclient.append(fullpath,
-          bufferSize.getValue(), null);
-      final FSDataOutputStream out = new FSDataOutputStream(dfsout, null,
-          dfsout.getInitialLen());
+      final FSDataOutputStream out = dfsclient.append(fullpath,
+          bufferSize.getValue(), null, null);
       try {
         IOUtils.copyBytes(in, out, bufferSize.getValue());
       } finally {