|
@@ -122,6 +122,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
|
|
|
private volatile boolean appendChunk = false; // appending to existing partial block
|
|
|
private long initialFileSize = 0; // at time of file open
|
|
|
private Progressable progress;
|
|
|
+ private short blockReplication; // replication factor of file
|
|
|
|
|
|
private class Packet {
|
|
|
ByteBuffer buffer; // only one of buf and buffer is non-null
|
|
@@ -1025,12 +1026,13 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
|
|
|
}
|
|
|
|
|
|
private DFSOutputStream(DFSClient dfsClient, String src, long blockSize, Progressable progress,
|
|
|
- int bytesPerChecksum) throws IOException {
|
|
|
+ int bytesPerChecksum, short replication) throws IOException {
|
|
|
super(new PureJavaCrc32(), bytesPerChecksum, 4);
|
|
|
this.dfsClient = dfsClient;
|
|
|
this.conf = dfsClient.conf;
|
|
|
this.src = src;
|
|
|
this.blockSize = blockSize;
|
|
|
+ this.blockReplication = replication;
|
|
|
this.progress = progress;
|
|
|
if (progress != null) {
|
|
|
DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "+src);
|
|
@@ -1055,7 +1057,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
|
|
|
boolean createParent, short replication, long blockSize, Progressable progress,
|
|
|
int buffersize, int bytesPerChecksum)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
- this(dfsClient, src, blockSize, progress, bytesPerChecksum);
|
|
|
+ this(dfsClient, src, blockSize, progress, bytesPerChecksum, replication);
|
|
|
|
|
|
computePacketChunkSize(dfsClient.writePacketSize, bytesPerChecksum);
|
|
|
|
|
@@ -1081,7 +1083,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
|
|
|
DFSOutputStream(DFSClient dfsClient, String src, int buffersize, Progressable progress,
|
|
|
LocatedBlock lastBlock, HdfsFileStatus stat,
|
|
|
int bytesPerChecksum) throws IOException {
|
|
|
- this(dfsClient, src, stat.getBlockSize(), progress, bytesPerChecksum);
|
|
|
+ this(dfsClient, src, stat.getBlockSize(), progress, bytesPerChecksum, stat.getReplication());
|
|
|
initialFileSize = stat.getLen(); // length of file when opened
|
|
|
|
|
|
//
|
|
@@ -1291,6 +1293,27 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
|
|
|
public synchronized void hsync() throws IOException {
|
|
|
hflush();
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns the number of replicas of current block. This can be different
|
|
|
+ * from the designated replication factor of the file because the NameNode
|
|
|
+ * does not replicate the block to which a client is currently writing to.
|
|
|
+ * The client continues to write to a block even if a few datanodes in the
|
|
|
+ * write pipeline have failed.
|
|
|
+ * @return the number of valid replicas of the current block
|
|
|
+ */
|
|
|
+ public synchronized int getNumCurrentReplicas() throws IOException {
|
|
|
+ dfsClient.checkOpen();
|
|
|
+ isClosed();
|
|
|
+ if (streamer == null) {
|
|
|
+ return blockReplication; // no pipeline, return repl factor of file
|
|
|
+ }
|
|
|
+ DatanodeInfo[] currentNodes = streamer.getNodes();
|
|
|
+ if (currentNodes == null) {
|
|
|
+ return blockReplication; // no pipeline, return repl factor of file
|
|
|
+ }
|
|
|
+ return currentNodes.length;
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* Waits till all existing data is flushed and confirmations
|
|
@@ -1446,4 +1469,4 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
|
|
|
return streamer.getAccessToken();
|
|
|
}
|
|
|
|
|
|
-}
|
|
|
+}
|