浏览代码

svn merge -c 1171136 from trunk for HDFS-2333.

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

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -1061,6 +1061,9 @@ Release 0.23.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)
+
   BREAKDOWN OF HDFS-1073 SUBTASKS
 
     HDFS-1521. Persist transaction ID on disk between NN restarts.

+ 11 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
@@ -79,7 +80,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -867,12 +867,20 @@ public class DFSClient implements 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
    * 
    * @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 = getFileInfo(src);

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -36,6 +36,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSOutputSummer;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -98,7 +99,8 @@ import org.apache.hadoop.util.PureJavaCrc32;
  * datanode from the original pipeline. The DataStreamer now
  * starts sending packets from the dataQueue.
 ****************************************************************/
-public class DFSOutputStream extends FSOutputSummer implements Syncable {
+@InterfaceAudience.Private
+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;
@@ -1537,7 +1539,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
    * write pipeline have failed. 
    * @return the number of valid replicas of the current block
    */
-  public synchronized int getNumCurrentReplicas() throws IOException {
+  synchronized int getNumCurrentReplicas() throws IOException {
     dfsClient.checkOpen();
     isClosed();
     if (streamer == null) {
@@ -1707,7 +1709,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
   /**
    * Returns the size of a file as it was when this stream was opened
    */
-  public long getInitialLen() {
+  long getInitialLen() {
     return initialFileSize;
   }
 

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

@@ -240,10 +240,8 @@ public class DistributedFileSystem extends FileSystem {
   @Override
   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);
   }
 
   @Override

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

@@ -47,7 +47,6 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
-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;
@@ -158,10 +157,8 @@ public class DatanodeWebHdfsMethods {
       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());
+      final FSDataOutputStream out = dfsclient.append(fullpath,
+          bufferSize.getValue(), null, null);
       try {
         IOUtils.copyBytes(in, out, bufferSize.getValue());
       } finally {