diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index f58a3f1c268..c7c5d114283 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/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. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index d1ba39fede4..1dc30fe4f70 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/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); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index 4ab8dfc54fc..3c64fae0bbf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/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; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index 18e5550381f..4de8fe4fd7e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/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 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java index e270a961bfc..14f492c90a0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java +++ b/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 {