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
This commit is contained in:
Tsz-wo Sze 2011-10-26 19:47:10 +00:00
parent 20a31036c8
commit 0c6c92e672
5 changed files with 22 additions and 14 deletions

View File

@ -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.

View File

@ -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);

View File

@ -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;
}

View File

@ -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

View File

@ -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 {