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

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1171136 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2011-09-15 15:28:23 +00:00
parent cb48bc1c93
commit 376a1a2511
5 changed files with 22 additions and 14 deletions

View File

@ -38,6 +38,9 @@ Trunk (unreleased changes)
HDFS-2331. Fix WebHdfsFileSystem compilation problems for a bug in JDK HDFS-2331. Fix WebHdfsFileSystem compilation problems for a bug in JDK
version < 1.6.0_26. (Abhijit Suresh Shingate via szetszwo) 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)
Release 0.23.0 - Unreleased Release 0.23.0 - Unreleased
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -42,6 +42,7 @@
import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.ContentSummary;
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.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FsServerDefaults; import org.apache.hadoop.fs.FsServerDefaults;
@ -79,7 +80,6 @@
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport; 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.NameNode;
import org.apache.hadoop.hdfs.server.namenode.SafeModeException; import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.DataOutputBuffer;
@ -797,12 +797,20 @@ private DFSOutputStream callAppend(HdfsFileStatus stat, String src,
* *
* @param src file name * @param src file name
* @param buffersize buffer size * @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 * @return an output stream for writing into the file
* *
* @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 { throws IOException {
checkOpen(); checkOpen();
HdfsFileStatus stat = getFileInfo(src); HdfsFileStatus stat = getFileInfo(src);

View File

@ -36,6 +36,7 @@
import java.util.List; import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSOutputSummer; import org.apache.hadoop.fs.FSOutputSummer;
import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileAlreadyExistsException;
@ -98,7 +99,8 @@
* datanode from the original pipeline. The DataStreamer now * datanode from the original pipeline. The DataStreamer now
* starts sending packets from the dataQueue. * 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 final DFSClient dfsClient;
private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB
private Socket s; private Socket s;
@ -1537,7 +1539,7 @@ public synchronized void hsync() throws IOException {
* write pipeline have failed. * write pipeline have failed.
* @return the number of valid replicas of the current block * @return the number of valid replicas of the current block
*/ */
public synchronized int getNumCurrentReplicas() throws IOException { synchronized int getNumCurrentReplicas() throws IOException {
dfsClient.checkOpen(); dfsClient.checkOpen();
isClosed(); isClosed();
if (streamer == null) { if (streamer == null) {
@ -1707,7 +1709,7 @@ synchronized void setTestFilename(String newname) {
/** /**
* Returns the size of a file as it was when this stream was opened * Returns the size of a file as it was when this stream was opened
*/ */
public long getInitialLen() { long getInitialLen() {
return initialFileSize; return initialFileSize;
} }

View File

@ -240,10 +240,8 @@ public FSDataInputStream open(Path f, int bufferSize) throws IOException {
@Override @Override
public FSDataOutputStream append(Path f, int bufferSize, public FSDataOutputStream append(Path f, int bufferSize,
Progressable progress) throws IOException { Progressable progress) throws IOException {
statistics.incrementWriteOps(1); statistics.incrementWriteOps(1);
final DFSOutputStream op = dfs.append(getPathName(f), bufferSize, progress); return dfs.append(getPathName(f), bufferSize, progress, statistics);
return new FSDataOutputStream(op, statistics, op.getInitialLen());
} }
@Override @Override

View File

@ -47,7 +47,6 @@
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream; 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.datanode.DataNode;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
@ -158,10 +157,8 @@ public Response post(
final Configuration conf = new Configuration(datanode.getConf()); final Configuration conf = new Configuration(datanode.getConf());
final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf); final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf);
final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf); final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
final DFSOutputStream dfsout = dfsclient.append(fullpath, final FSDataOutputStream out = dfsclient.append(fullpath,
bufferSize.getValue(), null); bufferSize.getValue(), null, null);
final FSDataOutputStream out = new FSDataOutputStream(dfsout, null,
dfsout.getInitialLen());
try { try {
IOUtils.copyBytes(in, out, bufferSize.getValue()); IOUtils.copyBytes(in, out, bufferSize.getValue());
} finally { } finally {