|
@@ -22,10 +22,14 @@ import java.io.InterruptedIOException;
|
|
import java.nio.ByteBuffer;
|
|
import java.nio.ByteBuffer;
|
|
import java.nio.channels.ClosedChannelException;
|
|
import java.nio.channels.ClosedChannelException;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
|
|
+import java.util.Arrays;
|
|
|
|
+import java.util.Collections;
|
|
import java.util.EnumSet;
|
|
import java.util.EnumSet;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.concurrent.BlockingQueue;
|
|
import java.util.concurrent.BlockingQueue;
|
|
import java.util.concurrent.LinkedBlockingQueue;
|
|
import java.util.concurrent.LinkedBlockingQueue;
|
|
|
|
+
|
|
|
|
+import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.fs.CreateFlag;
|
|
import org.apache.hadoop.fs.CreateFlag;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
@@ -59,12 +63,12 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
*/
|
|
*/
|
|
private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
|
|
private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
|
|
private ByteBuffer[] cellBuffers;
|
|
private ByteBuffer[] cellBuffers;
|
|
- private final short blockGroupBlocks = HdfsConstants.NUM_DATA_BLOCKS
|
|
|
|
|
|
+ private final short numAllBlocks = HdfsConstants.NUM_DATA_BLOCKS
|
|
+ HdfsConstants.NUM_PARITY_BLOCKS;
|
|
+ HdfsConstants.NUM_PARITY_BLOCKS;
|
|
- private final short blockGroupDataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
|
|
|
|
|
|
+ private final short numDataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
|
|
private int curIdx = 0;
|
|
private int curIdx = 0;
|
|
/* bytes written in current block group */
|
|
/* bytes written in current block group */
|
|
- private long currentBlockGroupBytes = 0;
|
|
|
|
|
|
+ //private long currentBlockGroupBytes = 0;
|
|
|
|
|
|
//TODO: Use ErasureCoder interface (HDFS-7781)
|
|
//TODO: Use ErasureCoder interface (HDFS-7781)
|
|
private RawErasureEncoder encoder;
|
|
private RawErasureEncoder encoder;
|
|
@@ -73,10 +77,6 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
return streamers.get(0);
|
|
return streamers.get(0);
|
|
}
|
|
}
|
|
|
|
|
|
- private long getBlockGroupSize() {
|
|
|
|
- return blockSize * HdfsConstants.NUM_DATA_BLOCKS;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/** Construct a new output stream for creating a file. */
|
|
/** Construct a new output stream for creating a file. */
|
|
DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
|
|
DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
|
|
EnumSet<CreateFlag> flag, Progressable progress,
|
|
EnumSet<CreateFlag> flag, Progressable progress,
|
|
@@ -84,15 +84,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
throws IOException {
|
|
throws IOException {
|
|
super(dfsClient, src, stat, flag, progress, checksum, favoredNodes);
|
|
super(dfsClient, src, stat, flag, progress, checksum, favoredNodes);
|
|
DFSClient.LOG.info("Creating striped output stream");
|
|
DFSClient.LOG.info("Creating striped output stream");
|
|
- if (blockGroupBlocks <= 1) {
|
|
|
|
- throw new IOException("The block group must contain more than one block.");
|
|
|
|
- }
|
|
|
|
|
|
+ checkConfiguration();
|
|
|
|
|
|
- cellBuffers = new ByteBuffer[blockGroupBlocks];
|
|
|
|
|
|
+ cellBuffers = new ByteBuffer[numAllBlocks];
|
|
List<BlockingQueue<LocatedBlock>> stripeBlocks = new ArrayList<>();
|
|
List<BlockingQueue<LocatedBlock>> stripeBlocks = new ArrayList<>();
|
|
|
|
|
|
- for (int i = 0; i < blockGroupBlocks; i++) {
|
|
|
|
- stripeBlocks.add(new LinkedBlockingQueue<LocatedBlock>(blockGroupBlocks));
|
|
|
|
|
|
+ for (int i = 0; i < numAllBlocks; i++) {
|
|
|
|
+ stripeBlocks.add(new LinkedBlockingQueue<LocatedBlock>(numAllBlocks));
|
|
try {
|
|
try {
|
|
cellBuffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize));
|
|
cellBuffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize));
|
|
} catch (InterruptedException ie) {
|
|
} catch (InterruptedException ie) {
|
|
@@ -103,29 +101,38 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
encoder = new RSRawEncoder();
|
|
encoder = new RSRawEncoder();
|
|
- encoder.initialize(blockGroupDataBlocks,
|
|
|
|
- blockGroupBlocks - blockGroupDataBlocks, cellSize);
|
|
|
|
|
|
+ encoder.initialize(numDataBlocks,
|
|
|
|
+ numAllBlocks - numDataBlocks, cellSize);
|
|
|
|
|
|
- streamers = new ArrayList<>(blockGroupBlocks);
|
|
|
|
- for (short i = 0; i < blockGroupBlocks; i++) {
|
|
|
|
|
|
+ List<StripedDataStreamer> s = new ArrayList<>(numAllBlocks);
|
|
|
|
+ for (short i = 0; i < numAllBlocks; i++) {
|
|
StripedDataStreamer streamer = new StripedDataStreamer(stat, null,
|
|
StripedDataStreamer streamer = new StripedDataStreamer(stat, null,
|
|
dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
|
|
dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
|
|
i, stripeBlocks);
|
|
i, stripeBlocks);
|
|
if (favoredNodes != null && favoredNodes.length != 0) {
|
|
if (favoredNodes != null && favoredNodes.length != 0) {
|
|
streamer.setFavoredNodes(favoredNodes);
|
|
streamer.setFavoredNodes(favoredNodes);
|
|
}
|
|
}
|
|
- streamers.add(streamer);
|
|
|
|
|
|
+ s.add(streamer);
|
|
}
|
|
}
|
|
|
|
+ streamers = Collections.unmodifiableList(s);
|
|
|
|
|
|
refreshStreamer();
|
|
refreshStreamer();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private void checkConfiguration() {
|
|
|
|
+ if (cellSize % bytesPerChecksum != 0) {
|
|
|
|
+ throw new HadoopIllegalArgumentException("Invalid values: "
|
|
|
|
+ + DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum
|
|
|
|
+ + ") must divide cell size (=" + cellSize + ").");
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
private void refreshStreamer() {
|
|
private void refreshStreamer() {
|
|
streamer = streamers.get(curIdx);
|
|
streamer = streamers.get(curIdx);
|
|
}
|
|
}
|
|
|
|
|
|
private void moveToNextStreamer() {
|
|
private void moveToNextStreamer() {
|
|
- curIdx = (curIdx + 1) % blockGroupBlocks;
|
|
|
|
|
|
+ curIdx = (curIdx + 1) % numAllBlocks;
|
|
refreshStreamer();
|
|
refreshStreamer();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -136,20 +143,21 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
* @param buffers data buffers + parity buffers
|
|
* @param buffers data buffers + parity buffers
|
|
*/
|
|
*/
|
|
private void encode(ByteBuffer[] buffers) {
|
|
private void encode(ByteBuffer[] buffers) {
|
|
- ByteBuffer[] dataBuffers = new ByteBuffer[blockGroupDataBlocks];
|
|
|
|
- ByteBuffer[] parityBuffers = new ByteBuffer[blockGroupBlocks - blockGroupDataBlocks];
|
|
|
|
- for (int i = 0; i < blockGroupBlocks; i++) {
|
|
|
|
- if (i < blockGroupDataBlocks) {
|
|
|
|
|
|
+ ByteBuffer[] dataBuffers = new ByteBuffer[numDataBlocks];
|
|
|
|
+ ByteBuffer[] parityBuffers = new ByteBuffer[numAllBlocks - numDataBlocks];
|
|
|
|
+ for (int i = 0; i < numAllBlocks; i++) {
|
|
|
|
+ if (i < numDataBlocks) {
|
|
dataBuffers[i] = buffers[i];
|
|
dataBuffers[i] = buffers[i];
|
|
} else {
|
|
} else {
|
|
- parityBuffers[i - blockGroupDataBlocks] = buffers[i];
|
|
|
|
|
|
+ parityBuffers[i - numDataBlocks] = buffers[i];
|
|
}
|
|
}
|
|
}
|
|
}
|
|
encoder.encode(dataBuffers, parityBuffers);
|
|
encoder.encode(dataBuffers, parityBuffers);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Generate packets from a given buffer
|
|
|
|
|
|
+ * Generate packets from a given buffer. This is only used for streamers
|
|
|
|
+ * writing parity blocks.
|
|
*
|
|
*
|
|
* @param byteBuffer the given buffer to generate packets
|
|
* @param byteBuffer the given buffer to generate packets
|
|
* @return packets generated
|
|
* @return packets generated
|
|
@@ -185,7 +193,6 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
throw new IOException(msg);
|
|
throw new IOException(msg);
|
|
}
|
|
}
|
|
|
|
|
|
-
|
|
|
|
// If current packet has not been enqueued for transmission,
|
|
// If current packet has not been enqueued for transmission,
|
|
// but the cell buffer is full, we need to enqueue the packet
|
|
// but the cell buffer is full, we need to enqueue the packet
|
|
if (currentPacket != null && getSizeOfCellnBuffer(curIdx) == cellSize) {
|
|
if (currentPacket != null && getSizeOfCellnBuffer(curIdx) == cellSize) {
|
|
@@ -213,13 +220,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
//When all data cells in a stripe are ready, we need to encode
|
|
//When all data cells in a stripe are ready, we need to encode
|
|
//them and generate some parity cells. These cells will be
|
|
//them and generate some parity cells. These cells will be
|
|
//converted to packets and put to their DataStreamer's queue.
|
|
//converted to packets and put to their DataStreamer's queue.
|
|
- if (curIdx == blockGroupDataBlocks) {
|
|
|
|
|
|
+ if (curIdx == numDataBlocks) {
|
|
//encode the data cells
|
|
//encode the data cells
|
|
- for (int k = 0; k < blockGroupDataBlocks; k++) {
|
|
|
|
|
|
+ for (int k = 0; k < numDataBlocks; k++) {
|
|
cellBuffers[k].flip();
|
|
cellBuffers[k].flip();
|
|
}
|
|
}
|
|
encode(cellBuffers);
|
|
encode(cellBuffers);
|
|
- for (int i = blockGroupDataBlocks; i < blockGroupBlocks; i++) {
|
|
|
|
|
|
+ for (int i = numDataBlocks; i < numAllBlocks; i++) {
|
|
ByteBuffer parityBuffer = cellBuffers[i];
|
|
ByteBuffer parityBuffer = cellBuffers[i];
|
|
List<DFSPacket> packets = generatePackets(parityBuffer);
|
|
List<DFSPacket> packets = generatePackets(parityBuffer);
|
|
for (DFSPacket p : packets) {
|
|
for (DFSPacket p : packets) {
|
|
@@ -245,13 +252,24 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
}
|
|
}
|
|
|
|
|
|
private void clearCellBuffers() {
|
|
private void clearCellBuffers() {
|
|
- for (int i = 0; i< blockGroupBlocks; i++) {
|
|
|
|
|
|
+ for (int i = 0; i< numAllBlocks; i++) {
|
|
cellBuffers[i].clear();
|
|
cellBuffers[i].clear();
|
|
|
|
+ if (i >= numDataBlocks) {
|
|
|
|
+ Arrays.fill(cellBuffers[i].array(), (byte) 0);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
private int stripeDataSize() {
|
|
private int stripeDataSize() {
|
|
- return blockGroupDataBlocks * cellSize;
|
|
|
|
|
|
+ return numDataBlocks * cellSize;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private long getCurrentBlockGroupBytes() {
|
|
|
|
+ long sum = 0;
|
|
|
|
+ for (int i = 0; i < numDataBlocks; i++) {
|
|
|
|
+ sum += streamers.get(i).getBytesCurBlock();
|
|
|
|
+ }
|
|
|
|
+ return sum;
|
|
}
|
|
}
|
|
|
|
|
|
private void notSupported(String headMsg)
|
|
private void notSupported(String headMsg)
|
|
@@ -270,7 +288,6 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
notSupported("hsync");
|
|
notSupported("hsync");
|
|
}
|
|
}
|
|
|
|
|
|
-
|
|
|
|
@Override
|
|
@Override
|
|
protected synchronized void start() {
|
|
protected synchronized void start() {
|
|
for (StripedDataStreamer streamer : streamers) {
|
|
for (StripedDataStreamer streamer : streamers) {
|
|
@@ -302,15 +319,11 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
// interrupt datastreamer if force is true
|
|
// interrupt datastreamer if force is true
|
|
@Override
|
|
@Override
|
|
protected void closeThreads(boolean force) throws IOException {
|
|
protected void closeThreads(boolean force) throws IOException {
|
|
- StripedDataStreamer leadingStreamer = null;
|
|
|
|
for (StripedDataStreamer streamer : streamers) {
|
|
for (StripedDataStreamer streamer : streamers) {
|
|
try {
|
|
try {
|
|
streamer.close(force);
|
|
streamer.close(force);
|
|
streamer.join();
|
|
streamer.join();
|
|
streamer.closeSocket();
|
|
streamer.closeSocket();
|
|
- if (streamer.isLeadingStreamer()) {
|
|
|
|
- leadingStreamer = streamer;
|
|
|
|
- }
|
|
|
|
} catch (InterruptedException e) {
|
|
} catch (InterruptedException e) {
|
|
throw new IOException("Failed to shutdown streamer");
|
|
throw new IOException("Failed to shutdown streamer");
|
|
} finally {
|
|
} finally {
|
|
@@ -318,40 +331,26 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
setClosed();
|
|
setClosed();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- assert leadingStreamer != null : "One streamer should be leader";
|
|
|
|
- leadingStreamer.countTailingBlockGroupBytes();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Override
|
|
|
|
- public synchronized void write(int b) throws IOException {
|
|
|
|
- super.write(b);
|
|
|
|
- currentBlockGroupBytes = (currentBlockGroupBytes + 1) % getBlockGroupSize();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @Override
|
|
|
|
- public synchronized void write(byte b[], int off, int len)
|
|
|
|
- throws IOException {
|
|
|
|
- super.write(b, off, len);
|
|
|
|
- currentBlockGroupBytes = (currentBlockGroupBytes + len) % getBlockGroupSize();
|
|
|
|
}
|
|
}
|
|
|
|
|
|
- private void writeParityCellsForLastStripe() throws IOException{
|
|
|
|
|
|
+ private void writeParityCellsForLastStripe() throws IOException {
|
|
|
|
+ final long currentBlockGroupBytes = getCurrentBlockGroupBytes();
|
|
long parityBlkSize = StripedBlockUtil.getInternalBlockLength(
|
|
long parityBlkSize = StripedBlockUtil.getInternalBlockLength(
|
|
- currentBlockGroupBytes, cellSize, blockGroupDataBlocks,
|
|
|
|
- blockGroupDataBlocks + 1);
|
|
|
|
|
|
+ currentBlockGroupBytes, cellSize, numDataBlocks,
|
|
|
|
+ numDataBlocks + 1);
|
|
if (parityBlkSize == 0 || currentBlockGroupBytes % stripeDataSize() == 0) {
|
|
if (parityBlkSize == 0 || currentBlockGroupBytes % stripeDataSize() == 0) {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
int parityCellSize = parityBlkSize % cellSize == 0 ? cellSize :
|
|
int parityCellSize = parityBlkSize % cellSize == 0 ? cellSize :
|
|
(int) (parityBlkSize % cellSize);
|
|
(int) (parityBlkSize % cellSize);
|
|
|
|
|
|
- for (int i = 0; i < blockGroupBlocks; i++) {
|
|
|
|
|
|
+ for (int i = 0; i < numAllBlocks; i++) {
|
|
long internalBlkLen = StripedBlockUtil.getInternalBlockLength(
|
|
long internalBlkLen = StripedBlockUtil.getInternalBlockLength(
|
|
- currentBlockGroupBytes, cellSize, blockGroupDataBlocks, i);
|
|
|
|
|
|
+ currentBlockGroupBytes, cellSize, numDataBlocks, i);
|
|
// Pad zero bytes to make all cells exactly the size of parityCellSize
|
|
// Pad zero bytes to make all cells exactly the size of parityCellSize
|
|
// If internal block is smaller than parity block, pad zero bytes.
|
|
// If internal block is smaller than parity block, pad zero bytes.
|
|
// Also pad zero bytes to all parity cells
|
|
// Also pad zero bytes to all parity cells
|
|
- if (internalBlkLen < parityBlkSize || i >= blockGroupDataBlocks) {
|
|
|
|
|
|
+ if (internalBlkLen < parityBlkSize || i >= numDataBlocks) {
|
|
int position = cellBuffers[i].position();
|
|
int position = cellBuffers[i].position();
|
|
assert position <= parityCellSize : "If an internal block is smaller" +
|
|
assert position <= parityCellSize : "If an internal block is smaller" +
|
|
" than parity block, then its last cell should be small than last" +
|
|
" than parity block, then its last cell should be small than last" +
|
|
@@ -365,9 +364,9 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
encode(cellBuffers);
|
|
encode(cellBuffers);
|
|
|
|
|
|
//write parity cells
|
|
//write parity cells
|
|
- curIdx = blockGroupDataBlocks;
|
|
|
|
|
|
+ curIdx = numDataBlocks;
|
|
refreshStreamer();
|
|
refreshStreamer();
|
|
- for (int i = blockGroupDataBlocks; i < blockGroupBlocks; i++) {
|
|
|
|
|
|
+ for (int i = numDataBlocks; i < numAllBlocks; i++) {
|
|
ByteBuffer parityBuffer = cellBuffers[i];
|
|
ByteBuffer parityBuffer = cellBuffers[i];
|
|
List<DFSPacket> packets = generatePackets(parityBuffer);
|
|
List<DFSPacket> packets = generatePackets(parityBuffer);
|
|
for (DFSPacket p : packets) {
|
|
for (DFSPacket p : packets) {
|
|
@@ -385,7 +384,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
@Override
|
|
@Override
|
|
void setClosed() {
|
|
void setClosed() {
|
|
super.setClosed();
|
|
super.setClosed();
|
|
- for (int i = 0; i < blockGroupBlocks; i++) {
|
|
|
|
|
|
+ for (int i = 0; i < numAllBlocks; i++) {
|
|
byteArrayManager.release(cellBuffers[i].array());
|
|
byteArrayManager.release(cellBuffers[i].array());
|
|
streamers.get(i).release();
|
|
streamers.get(i).release();
|
|
}
|
|
}
|
|
@@ -395,10 +394,11 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
protected synchronized void closeImpl() throws IOException {
|
|
protected synchronized void closeImpl() throws IOException {
|
|
if (isClosed()) {
|
|
if (isClosed()) {
|
|
IOException e = getLeadingStreamer().getLastException().getAndSet(null);
|
|
IOException e = getLeadingStreamer().getLastException().getAndSet(null);
|
|
- if (e == null)
|
|
|
|
- return;
|
|
|
|
- else
|
|
|
|
|
|
+ if (e != null) {
|
|
throw e;
|
|
throw e;
|
|
|
|
+ } else {
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
try {
|
|
try {
|
|
@@ -408,14 +408,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
streamer.waitAndQueuePacket(currentPacket);
|
|
streamer.waitAndQueuePacket(currentPacket);
|
|
currentPacket = null;
|
|
currentPacket = null;
|
|
}
|
|
}
|
|
- //if the last stripe is incomplete, generate and write parity cells
|
|
|
|
|
|
+ // if the last stripe is incomplete, generate and write parity cells
|
|
writeParityCellsForLastStripe();
|
|
writeParityCellsForLastStripe();
|
|
|
|
|
|
- for (int i = 0; i < blockGroupBlocks; i++) {
|
|
|
|
|
|
+ for (int i = 0; i < numAllBlocks; i++) {
|
|
curIdx = i;
|
|
curIdx = i;
|
|
refreshStreamer();
|
|
refreshStreamer();
|
|
- if (streamer.getBytesCurBlock()!= 0 ||
|
|
|
|
- currentBlockGroupBytes < getBlockGroupSize()) {
|
|
|
|
|
|
+ if (streamer.getBytesCurBlock() > 0) {
|
|
// send an empty packet to mark the end of the block
|
|
// send an empty packet to mark the end of the block
|
|
currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(),
|
|
currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(),
|
|
streamer.getAndIncCurrentSeqno(), true);
|
|
streamer.getAndIncCurrentSeqno(), true);
|
|
@@ -425,9 +424,8 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
flushInternal();
|
|
flushInternal();
|
|
}
|
|
}
|
|
|
|
|
|
- // get last block before destroying the streamer
|
|
|
|
- ExtendedBlock lastBlock = streamers.get(0).getBlock();
|
|
|
|
closeThreads(false);
|
|
closeThreads(false);
|
|
|
|
+ final ExtendedBlock lastBlock = getCommittedBlock();
|
|
TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER);
|
|
TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER);
|
|
try {
|
|
try {
|
|
completeFile(lastBlock);
|
|
completeFile(lastBlock);
|
|
@@ -435,10 +433,35 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
scope.close();
|
|
scope.close();
|
|
}
|
|
}
|
|
dfsClient.endFileLease(fileId);
|
|
dfsClient.endFileLease(fileId);
|
|
- } catch (ClosedChannelException e) {
|
|
|
|
|
|
+ } catch (ClosedChannelException ignored) {
|
|
} finally {
|
|
} finally {
|
|
setClosed();
|
|
setClosed();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Generate the block which is reported and will be committed in NameNode.
|
|
|
|
+ * Need to go through all the streamers writing data blocks and add their
|
|
|
|
+ * bytesCurBlock together. Note that at this time all streamers have been
|
|
|
|
+ * closed. Also this calculation can cover streamers with writing failures.
|
|
|
|
+ *
|
|
|
|
+ * @return An ExtendedBlock with size of the whole block group.
|
|
|
|
+ */
|
|
|
|
+ ExtendedBlock getCommittedBlock() throws IOException {
|
|
|
|
+ ExtendedBlock b = getLeadingStreamer().getBlock();
|
|
|
|
+ if (b == null) {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ final ExtendedBlock block = new ExtendedBlock(b);
|
|
|
|
+ final boolean atBlockGroupBoundary =
|
|
|
|
+ getLeadingStreamer().getBytesCurBlock() == 0 &&
|
|
|
|
+ getLeadingStreamer().getBlock() != null &&
|
|
|
|
+ getLeadingStreamer().getBlock().getNumBytes() > 0;
|
|
|
|
+ for (int i = 1; i < numDataBlocks; i++) {
|
|
|
|
+ block.setNumBytes(block.getNumBytes() +
|
|
|
|
+ (atBlockGroupBoundary ? streamers.get(i).getBlock().getNumBytes() :
|
|
|
|
+ streamers.get(i).getBytesCurBlock()));
|
|
|
|
+ }
|
|
|
|
+ return block;
|
|
|
|
+ }
|
|
}
|
|
}
|