|
@@ -28,14 +28,16 @@ import java.util.EnumSet;
|
|
|
import java.util.List;
|
|
|
import java.util.concurrent.BlockingQueue;
|
|
|
import java.util.concurrent.LinkedBlockingQueue;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.fs.CreateFlag;
|
|
|
-import org.apache.hadoop.hdfs.protocol.ECInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
+import org.apache.hadoop.io.MultipleIOException;
|
|
|
+import org.apache.hadoop.io.erasurecode.ECSchema;
|
|
|
import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
|
|
|
import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
|
|
|
import org.apache.hadoop.util.DataChecksum;
|
|
@@ -44,6 +46,8 @@ import org.apache.htrace.Sampler;
|
|
|
import org.apache.htrace.Trace;
|
|
|
import org.apache.htrace.TraceScope;
|
|
|
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
+
|
|
|
|
|
|
/****************************************************************
|
|
|
* The DFSStripedOutputStream class supports writing files in striped
|
|
@@ -55,33 +59,154 @@ import org.apache.htrace.TraceScope;
|
|
|
|
|
|
@InterfaceAudience.Private
|
|
|
public class DFSStripedOutputStream extends DFSOutputStream {
|
|
|
+ /** Coordinate the communication between the streamers. */
|
|
|
+ static class Coordinator {
|
|
|
+ private final List<BlockingQueue<ExtendedBlock>> endBlocks;
|
|
|
+ private final List<BlockingQueue<LocatedBlock>> stripedBlocks;
|
|
|
+ private volatile boolean shouldLocateFollowingBlock = false;
|
|
|
+
|
|
|
+ Coordinator(final int numDataBlocks, final int numAllBlocks) {
|
|
|
+ endBlocks = new ArrayList<>(numDataBlocks);
|
|
|
+ for (int i = 0; i < numDataBlocks; i++) {
|
|
|
+ endBlocks.add(new LinkedBlockingQueue<ExtendedBlock>(1));
|
|
|
+ }
|
|
|
|
|
|
- private final List<StripedDataStreamer> streamers;
|
|
|
- /**
|
|
|
- * Size of each striping cell, must be a multiple of bytesPerChecksum
|
|
|
- */
|
|
|
- private final ECInfo ecInfo;
|
|
|
- private final int cellSize;
|
|
|
- // checksum buffer, we only need to calculate checksum for parity blocks
|
|
|
- private byte[] checksumBuf;
|
|
|
- private ByteBuffer[] cellBuffers;
|
|
|
+ stripedBlocks = new ArrayList<>(numAllBlocks);
|
|
|
+ for (int i = 0; i < numAllBlocks; i++) {
|
|
|
+ stripedBlocks.add(new LinkedBlockingQueue<LocatedBlock>(1));
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- private final short numAllBlocks;
|
|
|
- private final short numDataBlocks;
|
|
|
+ boolean shouldLocateFollowingBlock() {
|
|
|
+ return shouldLocateFollowingBlock;
|
|
|
+ }
|
|
|
|
|
|
- private int curIdx = 0;
|
|
|
- /* bytes written in current block group */
|
|
|
- //private long currentBlockGroupBytes = 0;
|
|
|
+ void putEndBlock(int i, ExtendedBlock block) {
|
|
|
+ shouldLocateFollowingBlock = true;
|
|
|
|
|
|
- //TODO: Use ErasureCoder interface (HDFS-7781)
|
|
|
- private RawErasureEncoder encoder;
|
|
|
+ final boolean b = endBlocks.get(i).offer(block);
|
|
|
+ Preconditions.checkState(b, "Failed to add " + block
|
|
|
+ + " to endBlocks queue, i=" + i);
|
|
|
+ }
|
|
|
|
|
|
- private StripedDataStreamer getLeadingStreamer() {
|
|
|
- return streamers.get(0);
|
|
|
+ ExtendedBlock getEndBlock(int i) throws InterruptedIOException {
|
|
|
+ try {
|
|
|
+ return endBlocks.get(i).poll(30, TimeUnit.SECONDS);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ throw DFSUtil.toInterruptedIOException(
|
|
|
+ "getEndBlock interrupted, i=" + i, e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ void setBytesEndBlock(int i, long newBytes, ExtendedBlock block) {
|
|
|
+ ExtendedBlock b = endBlocks.get(i).peek();
|
|
|
+ if (b == null) {
|
|
|
+ // streamer just has failed, put end block and continue
|
|
|
+ b = block;
|
|
|
+ putEndBlock(i, b);
|
|
|
+ }
|
|
|
+ b.setNumBytes(newBytes);
|
|
|
+ }
|
|
|
+
|
|
|
+ void putStripedBlock(int i, LocatedBlock block) throws IOException {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("putStripedBlock " + block + ", i=" + i);
|
|
|
+ }
|
|
|
+ final boolean b = stripedBlocks.get(i).offer(block);
|
|
|
+ if (!b) {
|
|
|
+ throw new IOException("Failed: " + block + ", i=" + i);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ LocatedBlock getStripedBlock(int i) throws IOException {
|
|
|
+ final LocatedBlock lb;
|
|
|
+ try {
|
|
|
+ lb = stripedBlocks.get(i).poll(90, TimeUnit.SECONDS);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ throw DFSUtil.toInterruptedIOException("getStripedBlock interrupted", e);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (lb == null) {
|
|
|
+ throw new IOException("Failed: i=" + i);
|
|
|
+ }
|
|
|
+ return lb;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- private long getBlockGroupSize() {
|
|
|
- return blockSize * numDataBlocks;
|
|
|
+ /** Buffers for writing the data and parity cells of a strip. */
|
|
|
+ class CellBuffers {
|
|
|
+ private final ByteBuffer[] buffers;
|
|
|
+ private final byte[][] checksumArrays;
|
|
|
+
|
|
|
+ CellBuffers(int numParityBlocks) throws InterruptedException{
|
|
|
+ if (cellSize % bytesPerChecksum != 0) {
|
|
|
+ throw new HadoopIllegalArgumentException("Invalid values: "
|
|
|
+ + DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (="
|
|
|
+ + bytesPerChecksum + ") must divide cell size (=" + cellSize + ").");
|
|
|
+ }
|
|
|
+
|
|
|
+ checksumArrays = new byte[numParityBlocks][];
|
|
|
+ final int size = getChecksumSize() * (cellSize / bytesPerChecksum);
|
|
|
+ for (int i = 0; i < checksumArrays.length; i++) {
|
|
|
+ checksumArrays[i] = new byte[size];
|
|
|
+ }
|
|
|
+
|
|
|
+ buffers = new ByteBuffer[numAllBlocks];
|
|
|
+ for (int i = 0; i < buffers.length; i++) {
|
|
|
+ buffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private ByteBuffer[] getBuffers() {
|
|
|
+ return buffers;
|
|
|
+ }
|
|
|
+
|
|
|
+ byte[] getChecksumArray(int i) {
|
|
|
+ return checksumArrays[i - numDataBlocks];
|
|
|
+ }
|
|
|
+
|
|
|
+ private int addTo(int i, byte[] b, int off, int len) {
|
|
|
+ final ByteBuffer buf = buffers[i];
|
|
|
+ final int pos = buf.position() + len;
|
|
|
+ Preconditions.checkState(pos <= cellSize);
|
|
|
+ buf.put(b, off, len);
|
|
|
+ return pos;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void clear() {
|
|
|
+ for (int i = 0; i< numAllBlocks; i++) {
|
|
|
+ buffers[i].clear();
|
|
|
+ if (i >= numDataBlocks) {
|
|
|
+ Arrays.fill(buffers[i].array(), (byte) 0);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void release() {
|
|
|
+ for (int i = 0; i < numAllBlocks; i++) {
|
|
|
+ byteArrayManager.release(buffers[i].array());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void flipDataBuffers() {
|
|
|
+ for (int i = 0; i < numDataBlocks; i++) {
|
|
|
+ buffers[i].flip();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private final Coordinator coordinator;
|
|
|
+ private final CellBuffers cellBuffers;
|
|
|
+ private final RawErasureEncoder encoder;
|
|
|
+ private final List<StripedDataStreamer> streamers;
|
|
|
+
|
|
|
+ /** Size of each striping cell, must be a multiple of bytesPerChecksum */
|
|
|
+ private final int cellSize;
|
|
|
+ private final int numAllBlocks;
|
|
|
+ private final int numDataBlocks;
|
|
|
+
|
|
|
+ private StripedDataStreamer getLeadingStreamer() {
|
|
|
+ return streamers.get(0);
|
|
|
}
|
|
|
|
|
|
/** Construct a new output stream for creating a file. */
|
|
@@ -90,82 +215,94 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
|
DataChecksum checksum, String[] favoredNodes)
|
|
|
throws IOException {
|
|
|
super(dfsClient, src, stat, flag, progress, checksum, favoredNodes);
|
|
|
- DFSClient.LOG.info("Creating striped output stream");
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Creating DFSStripedOutputStream for " + src);
|
|
|
+ }
|
|
|
|
|
|
// ECInfo is restored from NN just before writing striped files.
|
|
|
- ecInfo = dfsClient.getErasureCodingInfo(src);
|
|
|
- cellSize = ecInfo.getSchema().getChunkSize();
|
|
|
- numAllBlocks = (short)(ecInfo.getSchema().getNumDataUnits()
|
|
|
- + ecInfo.getSchema().getNumParityUnits());
|
|
|
- numDataBlocks = (short)ecInfo.getSchema().getNumDataUnits();
|
|
|
+ //TODO reduce an rpc call HDFS-8289
|
|
|
+ final ECSchema schema = dfsClient.getErasureCodingInfo(src).getSchema();
|
|
|
+ final int numParityBlocks = schema.getNumParityUnits();
|
|
|
+ cellSize = schema.getChunkSize();
|
|
|
+ numDataBlocks = schema.getNumDataUnits();
|
|
|
+ numAllBlocks = numDataBlocks + numParityBlocks;
|
|
|
|
|
|
- checkConfiguration();
|
|
|
-
|
|
|
- checksumBuf = new byte[getChecksumSize() * (cellSize / bytesPerChecksum)];
|
|
|
- cellBuffers = new ByteBuffer[numAllBlocks];
|
|
|
- List<BlockingQueue<LocatedBlock>> stripeBlocks = new ArrayList<>();
|
|
|
+ encoder = new RSRawEncoder();
|
|
|
+ encoder.initialize(numDataBlocks, numParityBlocks, cellSize);
|
|
|
|
|
|
- for (int i = 0; i < numAllBlocks; i++) {
|
|
|
- stripeBlocks.add(new LinkedBlockingQueue<LocatedBlock>(numAllBlocks));
|
|
|
- try {
|
|
|
- cellBuffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize));
|
|
|
- } catch (InterruptedException ie) {
|
|
|
- final InterruptedIOException iioe = new InterruptedIOException(
|
|
|
- "create cell buffers");
|
|
|
- iioe.initCause(ie);
|
|
|
- throw iioe;
|
|
|
- }
|
|
|
+ coordinator = new Coordinator(numDataBlocks, numAllBlocks);
|
|
|
+ try {
|
|
|
+ cellBuffers = new CellBuffers(numParityBlocks);
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ throw DFSUtil.toInterruptedIOException("Failed to create cell buffers", ie);
|
|
|
}
|
|
|
- encoder = new RSRawEncoder();
|
|
|
- encoder.initialize(numDataBlocks,
|
|
|
- numAllBlocks - numDataBlocks, cellSize);
|
|
|
|
|
|
List<StripedDataStreamer> s = new ArrayList<>(numAllBlocks);
|
|
|
for (short i = 0; i < numAllBlocks; i++) {
|
|
|
- StripedDataStreamer streamer = new StripedDataStreamer(stat, null,
|
|
|
+ StripedDataStreamer streamer = new StripedDataStreamer(stat,
|
|
|
dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
|
|
|
- i, stripeBlocks, favoredNodes);
|
|
|
+ favoredNodes, i, coordinator);
|
|
|
s.add(streamer);
|
|
|
}
|
|
|
streamers = Collections.unmodifiableList(s);
|
|
|
+ setCurrentStreamer(0);
|
|
|
+ }
|
|
|
|
|
|
- refreshStreamer();
|
|
|
+ StripedDataStreamer getStripedDataStreamer(int i) {
|
|
|
+ return streamers.get(i);
|
|
|
}
|
|
|
|
|
|
- private void checkConfiguration() {
|
|
|
- if (cellSize % bytesPerChecksum != 0) {
|
|
|
- throw new HadoopIllegalArgumentException("Invalid values: "
|
|
|
- + DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum
|
|
|
- + ") must divide cell size (=" + cellSize + ").");
|
|
|
- }
|
|
|
+ int getCurrentIndex() {
|
|
|
+ return getCurrentStreamer().getIndex();
|
|
|
}
|
|
|
|
|
|
- private void refreshStreamer() {
|
|
|
- streamer = streamers.get(curIdx);
|
|
|
+ StripedDataStreamer getCurrentStreamer() {
|
|
|
+ return (StripedDataStreamer)streamer;
|
|
|
}
|
|
|
|
|
|
- private void moveToNextStreamer() {
|
|
|
- curIdx = (curIdx + 1) % numAllBlocks;
|
|
|
- refreshStreamer();
|
|
|
+ private StripedDataStreamer setCurrentStreamer(int i) {
|
|
|
+ streamer = streamers.get(i);
|
|
|
+ return getCurrentStreamer();
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * encode the buffers.
|
|
|
- * After encoding, flip each buffer.
|
|
|
+ * Encode the buffers, i.e. compute parities.
|
|
|
*
|
|
|
* @param buffers data buffers + parity buffers
|
|
|
*/
|
|
|
- private void encode(ByteBuffer[] buffers) {
|
|
|
- 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];
|
|
|
- } else {
|
|
|
- parityBuffers[i - numDataBlocks] = buffers[i];
|
|
|
+ private static void encode(RawErasureEncoder encoder, int numData,
|
|
|
+ ByteBuffer[] buffers) {
|
|
|
+ final ByteBuffer[] dataBuffers = new ByteBuffer[numData];
|
|
|
+ final ByteBuffer[] parityBuffers = new ByteBuffer[buffers.length - numData];
|
|
|
+ System.arraycopy(buffers, 0, dataBuffers, 0, dataBuffers.length);
|
|
|
+ System.arraycopy(buffers, numData, parityBuffers, 0, parityBuffers.length);
|
|
|
+
|
|
|
+ encoder.encode(dataBuffers, parityBuffers);
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ private void checkStreamers() throws IOException {
|
|
|
+ int count = 0;
|
|
|
+ for(StripedDataStreamer s : streamers) {
|
|
|
+ if (!s.isFailed()) {
|
|
|
+ count++;
|
|
|
}
|
|
|
}
|
|
|
- encoder.encode(dataBuffers, parityBuffers);
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("checkStreamers: " + streamers);
|
|
|
+ LOG.debug("count=" + count);
|
|
|
+ }
|
|
|
+ if (count < numDataBlocks) {
|
|
|
+ throw new IOException("Failed: the number of remaining blocks = "
|
|
|
+ + count + " < the number of data blocks = " + numDataBlocks);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void handleStreamerFailure(String err, Exception e) throws IOException {
|
|
|
+ LOG.warn("Failed: " + err + ", " + this, e);
|
|
|
+ getCurrentStreamer().setIsFailed(true);
|
|
|
+ checkStreamers();
|
|
|
+ currentPacket = null;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -173,11 +310,12 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
|
* writing parity blocks.
|
|
|
*
|
|
|
* @param byteBuffer the given buffer to generate packets
|
|
|
+ * @param checksumBuf the checksum buffer
|
|
|
* @return packets generated
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- private List<DFSPacket> generatePackets(ByteBuffer byteBuffer)
|
|
|
- throws IOException{
|
|
|
+ private List<DFSPacket> generatePackets(
|
|
|
+ ByteBuffer byteBuffer, byte[] checksumBuf) throws IOException{
|
|
|
List<DFSPacket> packets = new ArrayList<>();
|
|
|
assert byteBuffer.hasArray();
|
|
|
getDataChecksum().calculateChunkedSums(byteBuffer.array(), 0,
|
|
@@ -201,82 +339,47 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- protected synchronized void writeChunk(byte[] b, int offset, int len,
|
|
|
+ protected synchronized void writeChunk(byte[] bytes, int offset, int len,
|
|
|
byte[] checksum, int ckoff, int cklen) throws IOException {
|
|
|
- super.writeChunk(b, offset, len, checksum, ckoff, cklen);
|
|
|
-
|
|
|
- if (getSizeOfCellnBuffer(curIdx) <= cellSize) {
|
|
|
- addToCellBuffer(b, offset, len);
|
|
|
- } else {
|
|
|
- String msg = "Writing a chunk should not overflow the cell buffer.";
|
|
|
- DFSClient.LOG.info(msg);
|
|
|
- throw new IOException(msg);
|
|
|
- }
|
|
|
-
|
|
|
- // If current packet has not been enqueued for transmission,
|
|
|
- // but the cell buffer is full, we need to enqueue the packet
|
|
|
- if (currentPacket != null && getSizeOfCellnBuffer(curIdx) == cellSize) {
|
|
|
- if (DFSClient.LOG.isDebugEnabled()) {
|
|
|
- DFSClient.LOG.debug("DFSClient writeChunk cell buffer full seqno=" +
|
|
|
- currentPacket.getSeqno() +
|
|
|
- ", curIdx=" + curIdx +
|
|
|
- ", src=" + src +
|
|
|
- ", bytesCurBlock=" + streamer.getBytesCurBlock() +
|
|
|
- ", blockSize=" + blockSize +
|
|
|
- ", appendChunk=" + streamer.getAppendChunk());
|
|
|
+ final int index = getCurrentIndex();
|
|
|
+ final StripedDataStreamer current = getCurrentStreamer();
|
|
|
+ final int pos = cellBuffers.addTo(index, bytes, offset, len);
|
|
|
+ final boolean cellFull = pos == cellSize;
|
|
|
+
|
|
|
+ final long oldBytes = current.getBytesCurBlock();
|
|
|
+ if (!current.isFailed()) {
|
|
|
+ try {
|
|
|
+ super.writeChunk(bytes, offset, len, checksum, ckoff, cklen);
|
|
|
+
|
|
|
+ // cell is full and current packet has not been enqueued,
|
|
|
+ if (cellFull && currentPacket != null) {
|
|
|
+ enqueueCurrentPacketFull();
|
|
|
+ }
|
|
|
+ } catch(Exception e) {
|
|
|
+ handleStreamerFailure("offset=" + offset + ", length=" + len, e);
|
|
|
}
|
|
|
- streamer.waitAndQueuePacket(currentPacket);
|
|
|
- currentPacket = null;
|
|
|
- adjustChunkBoundary();
|
|
|
- endBlock();
|
|
|
+ }
|
|
|
+
|
|
|
+ if (current.isFailed()) {
|
|
|
+ final long newBytes = oldBytes + len;
|
|
|
+ coordinator.setBytesEndBlock(index, newBytes, current.getBlock());
|
|
|
+ current.setBytesCurBlock(newBytes);
|
|
|
}
|
|
|
|
|
|
// Two extra steps are needed when a striping cell is full:
|
|
|
// 1. Forward the current index pointer
|
|
|
// 2. Generate parity packets if a full stripe of data cells are present
|
|
|
- if (getSizeOfCellnBuffer(curIdx) == cellSize) {
|
|
|
- //move curIdx to next cell
|
|
|
- moveToNextStreamer();
|
|
|
+ if (cellFull) {
|
|
|
+ int next = index + 1;
|
|
|
//When all data cells in a stripe are ready, we need to encode
|
|
|
//them and generate some parity cells. These cells will be
|
|
|
//converted to packets and put to their DataStreamer's queue.
|
|
|
- if (curIdx == numDataBlocks) {
|
|
|
- //encode the data cells
|
|
|
- for (int k = 0; k < numDataBlocks; k++) {
|
|
|
- cellBuffers[k].flip();
|
|
|
- }
|
|
|
- encode(cellBuffers);
|
|
|
- for (int i = numDataBlocks; i < numAllBlocks; i++) {
|
|
|
- ByteBuffer parityBuffer = cellBuffers[i];
|
|
|
- List<DFSPacket> packets = generatePackets(parityBuffer);
|
|
|
- for (DFSPacket p : packets) {
|
|
|
- currentPacket = p;
|
|
|
- streamer.waitAndQueuePacket(currentPacket);
|
|
|
- currentPacket = null;
|
|
|
- }
|
|
|
- endBlock();
|
|
|
- moveToNextStreamer();
|
|
|
- }
|
|
|
- //read next stripe to cellBuffers
|
|
|
- clearCellBuffers();
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void addToCellBuffer(byte[] b, int off, int len) {
|
|
|
- cellBuffers[curIdx].put(b, off, len);
|
|
|
- }
|
|
|
-
|
|
|
- private int getSizeOfCellnBuffer(int cellIndex) {
|
|
|
- return cellBuffers[cellIndex].position();
|
|
|
- }
|
|
|
-
|
|
|
- private void clearCellBuffers() {
|
|
|
- for (int i = 0; i< numAllBlocks; i++) {
|
|
|
- cellBuffers[i].clear();
|
|
|
- if (i >= numDataBlocks) {
|
|
|
- Arrays.fill(cellBuffers[i].array(), (byte) 0);
|
|
|
+ if (next == numDataBlocks) {
|
|
|
+ cellBuffers.flipDataBuffers();
|
|
|
+ writeParityCells();
|
|
|
+ next = 0;
|
|
|
}
|
|
|
+ setCurrentStreamer(next);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -284,20 +387,14 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
|
return numDataBlocks * cellSize;
|
|
|
}
|
|
|
|
|
|
- private void notSupported(String headMsg)
|
|
|
- throws IOException{
|
|
|
- throw new IOException(
|
|
|
- headMsg + " is now not supported for striping layout.");
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
- public void hflush() throws IOException {
|
|
|
- notSupported("hflush");
|
|
|
+ public void hflush() {
|
|
|
+ throw new UnsupportedOperationException();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void hsync() throws IOException {
|
|
|
- notSupported("hsync");
|
|
|
+ public void hsync() {
|
|
|
+ throw new UnsupportedOperationException();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -327,29 +424,28 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
|
return closed || getLeadingStreamer().streamerClosed();
|
|
|
}
|
|
|
|
|
|
- // shutdown datastreamer and responseprocessor threads.
|
|
|
- // interrupt datastreamer if force is true
|
|
|
@Override
|
|
|
protected void closeThreads(boolean force) throws IOException {
|
|
|
- int index = 0;
|
|
|
- boolean exceptionOccurred = false;
|
|
|
+ final MultipleIOException.Builder b = new MultipleIOException.Builder();
|
|
|
for (StripedDataStreamer streamer : streamers) {
|
|
|
try {
|
|
|
streamer.close(force);
|
|
|
streamer.join();
|
|
|
streamer.closeSocket();
|
|
|
- } catch (InterruptedException | IOException e) {
|
|
|
- DFSClient.LOG.error("Failed to shutdown streamer: name="
|
|
|
- + streamer.getName() + ", index=" + index + ", file=" + src, e);
|
|
|
- exceptionOccurred = true;
|
|
|
+ } catch(Exception e) {
|
|
|
+ try {
|
|
|
+ handleStreamerFailure("force=" + force, e);
|
|
|
+ } catch(IOException ioe) {
|
|
|
+ b.add(ioe);
|
|
|
+ }
|
|
|
} finally {
|
|
|
streamer.setSocketToNull();
|
|
|
setClosed();
|
|
|
- index++;
|
|
|
}
|
|
|
}
|
|
|
- if (exceptionOccurred) {
|
|
|
- throw new IOException("Failed to shutdown streamer");
|
|
|
+ final IOException ioe = b.build();
|
|
|
+ if (ioe != null) {
|
|
|
+ throw ioe;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -370,50 +466,69 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
|
if (currentBlockGroupBytes % stripeDataSize() == 0) {
|
|
|
return;
|
|
|
}
|
|
|
- long firstCellSize = getLeadingStreamer().getBytesCurBlock() % cellSize;
|
|
|
- long parityCellSize = firstCellSize > 0 && firstCellSize < cellSize ?
|
|
|
+
|
|
|
+ final int firstCellSize = (int)(getStripedDataStreamer(0).getBytesCurBlock() % cellSize);
|
|
|
+ final int parityCellSize = firstCellSize > 0 && firstCellSize < cellSize?
|
|
|
firstCellSize : cellSize;
|
|
|
+ final ByteBuffer[] buffers = cellBuffers.getBuffers();
|
|
|
|
|
|
for (int i = 0; i < numAllBlocks; i++) {
|
|
|
// Pad zero bytes to make all cells exactly the size of parityCellSize
|
|
|
// If internal block is smaller than parity block, pad zero bytes.
|
|
|
// Also pad zero bytes to all parity cells
|
|
|
- int position = cellBuffers[i].position();
|
|
|
+ final int position = buffers[i].position();
|
|
|
assert position <= parityCellSize : "If an internal block is smaller" +
|
|
|
" than parity block, then its last cell should be small than last" +
|
|
|
" parity cell";
|
|
|
for (int j = 0; j < parityCellSize - position; j++) {
|
|
|
- cellBuffers[i].put((byte) 0);
|
|
|
+ buffers[i].put((byte) 0);
|
|
|
}
|
|
|
- cellBuffers[i].flip();
|
|
|
+ buffers[i].flip();
|
|
|
}
|
|
|
- encode(cellBuffers);
|
|
|
|
|
|
- // write parity cells
|
|
|
- curIdx = numDataBlocks;
|
|
|
- refreshStreamer();
|
|
|
+ writeParityCells();
|
|
|
+ }
|
|
|
+
|
|
|
+ void writeParityCells() throws IOException {
|
|
|
+ final ByteBuffer[] buffers = cellBuffers.getBuffers();
|
|
|
+ //encode the data cells
|
|
|
+ encode(encoder, numDataBlocks, buffers);
|
|
|
for (int i = numDataBlocks; i < numAllBlocks; i++) {
|
|
|
- ByteBuffer parityBuffer = cellBuffers[i];
|
|
|
- List<DFSPacket> packets = generatePackets(parityBuffer);
|
|
|
- for (DFSPacket p : packets) {
|
|
|
- currentPacket = p;
|
|
|
- streamer.waitAndQueuePacket(currentPacket);
|
|
|
- currentPacket = null;
|
|
|
+ writeParity(i, buffers[i], cellBuffers.getChecksumArray(i));
|
|
|
+ }
|
|
|
+ cellBuffers.clear();
|
|
|
+ }
|
|
|
+
|
|
|
+ void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf
|
|
|
+ ) throws IOException {
|
|
|
+ final StripedDataStreamer current = setCurrentStreamer(index);
|
|
|
+ final int len = buffer.limit();
|
|
|
+
|
|
|
+ final long oldBytes = current.getBytesCurBlock();
|
|
|
+ if (!current.isFailed()) {
|
|
|
+ try {
|
|
|
+ for (DFSPacket p : generatePackets(buffer, checksumBuf)) {
|
|
|
+ streamer.waitAndQueuePacket(p);
|
|
|
+ }
|
|
|
+ endBlock();
|
|
|
+ } catch(Exception e) {
|
|
|
+ handleStreamerFailure("oldBytes=" + oldBytes + ", len=" + len, e);
|
|
|
}
|
|
|
- endBlock();
|
|
|
- moveToNextStreamer();
|
|
|
}
|
|
|
|
|
|
- clearCellBuffers();
|
|
|
+ if (current.isFailed()) {
|
|
|
+ final long newBytes = oldBytes + len;
|
|
|
+ current.setBytesCurBlock(newBytes);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
void setClosed() {
|
|
|
super.setClosed();
|
|
|
for (int i = 0; i < numAllBlocks; i++) {
|
|
|
- byteArrayManager.release(cellBuffers[i].array());
|
|
|
streamers.get(i).release();
|
|
|
}
|
|
|
+ cellBuffers.release();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -425,25 +540,31 @@ public class DFSStripedOutputStream extends DFSOutputStream {
|
|
|
|
|
|
try {
|
|
|
// flush from all upper layers
|
|
|
- flushBuffer();
|
|
|
- if (currentPacket != null) {
|
|
|
- streamer.waitAndQueuePacket(currentPacket);
|
|
|
- currentPacket = null;
|
|
|
+ try {
|
|
|
+ flushBuffer();
|
|
|
+ if (currentPacket != null) {
|
|
|
+ enqueueCurrentPacket();
|
|
|
+ }
|
|
|
+ } catch(Exception e) {
|
|
|
+ handleStreamerFailure("closeImpl", e);
|
|
|
}
|
|
|
+
|
|
|
// if the last stripe is incomplete, generate and write parity cells
|
|
|
writeParityCellsForLastStripe();
|
|
|
|
|
|
for (int i = 0; i < numAllBlocks; i++) {
|
|
|
- curIdx = i;
|
|
|
- refreshStreamer();
|
|
|
- if (streamer.getBytesCurBlock() > 0) {
|
|
|
- // send an empty packet to mark the end of the block
|
|
|
- currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(),
|
|
|
- streamer.getAndIncCurrentSeqno(), true);
|
|
|
- currentPacket.setSyncBlock(shouldSyncBlock);
|
|
|
+ final StripedDataStreamer s = setCurrentStreamer(i);
|
|
|
+ if (!s.isFailed()) {
|
|
|
+ try {
|
|
|
+ if (s.getBytesCurBlock() > 0) {
|
|
|
+ setCurrentPacket2Empty();
|
|
|
+ }
|
|
|
+ // flush all data to Datanode
|
|
|
+ flushInternal();
|
|
|
+ } catch(Exception e) {
|
|
|
+ handleStreamerFailure("closeImpl", e);
|
|
|
+ }
|
|
|
}
|
|
|
- // flush all data to Datanode
|
|
|
- flushInternal();
|
|
|
}
|
|
|
|
|
|
closeThreads(false);
|