|
@@ -37,15 +37,13 @@ import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.io.OutputStream;
|
|
|
+import java.nio.Buffer;
|
|
|
import java.nio.ByteBuffer;
|
|
|
import java.util.UUID;
|
|
|
import java.util.List;
|
|
|
import java.util.ArrayList;
|
|
|
-import java.util.concurrent.CompletableFuture;
|
|
|
-import java.util.concurrent.ExecutorService;
|
|
|
-import java.util.concurrent.TimeoutException;
|
|
|
-import java.util.concurrent.ExecutionException;
|
|
|
-import java.util.concurrent.Executors;
|
|
|
+import java.util.concurrent.*;
|
|
|
+
|
|
|
import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls
|
|
|
.putBlockAsync;
|
|
|
import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls
|
|
@@ -84,25 +82,30 @@ public class ChunkOutputStream extends OutputStream {
|
|
|
private final long streamBufferFlushSize;
|
|
|
private final long streamBufferMaxSize;
|
|
|
private final long watchTimeout;
|
|
|
- private ByteBuffer buffer;
|
|
|
+ private List<ByteBuffer> bufferList;
|
|
|
// The IOException will be set by response handling thread in case there is an
|
|
|
// exception received in the response. If the exception is set, the next
|
|
|
// request will fail upfront.
|
|
|
private IOException ioException;
|
|
|
private ExecutorService responseExecutor;
|
|
|
|
|
|
- // position of the buffer where the last flush was attempted
|
|
|
- private int lastFlushPos;
|
|
|
+ // the effective length of data flushed so far
|
|
|
+ private long totalDataFlushedLength;
|
|
|
+
|
|
|
+ // effective data write attempted so far for the block
|
|
|
+ private long writtenDataLength;
|
|
|
|
|
|
- // position of the buffer till which the flush was successfully
|
|
|
- // acknowledged by all nodes in pipeline
|
|
|
- private int lastSuccessfulFlushIndex;
|
|
|
+ // total data which has been successfully flushed and acknowledged
|
|
|
+ // by all servers
|
|
|
+ private long totalAckDataLength;
|
|
|
|
|
|
// list to hold up all putBlock futures
|
|
|
private List<CompletableFuture<ContainerProtos.ContainerCommandResponseProto>>
|
|
|
futureList;
|
|
|
- // list maintaining commit indexes for putBlocks
|
|
|
- private List<Long> commitIndexList;
|
|
|
+ // map containing mapping for putBlock logIndex to to flushedDataLength Map.
|
|
|
+ private ConcurrentHashMap<Long, Long> commitIndex2flushedDataMap;
|
|
|
+
|
|
|
+ private int currentBufferIndex;
|
|
|
|
|
|
/**
|
|
|
* Creates a new ChunkOutputStream.
|
|
@@ -113,12 +116,17 @@ public class ChunkOutputStream extends OutputStream {
|
|
|
* @param xceiverClient client to perform container calls
|
|
|
* @param traceID container protocol call args
|
|
|
* @param chunkSize chunk size
|
|
|
+ * @param bufferList list of byte buffers
|
|
|
+ * @param streamBufferFlushSize flush size
|
|
|
+ * @param streamBufferMaxSize max size of the currentBuffer
|
|
|
+ * @param watchTimeout watch timeout
|
|
|
+ * @param checksum checksum
|
|
|
*/
|
|
|
public ChunkOutputStream(BlockID blockID, String key,
|
|
|
XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient,
|
|
|
String traceID, int chunkSize, long streamBufferFlushSize,
|
|
|
- long streamBufferMaxSize, long watchTimeout, ByteBuffer buffer,
|
|
|
- Checksum checksum) {
|
|
|
+ long streamBufferMaxSize, long watchTimeout,
|
|
|
+ List<ByteBuffer> bufferList, Checksum checksum) {
|
|
|
this.blockID = blockID;
|
|
|
this.key = key;
|
|
|
this.traceID = traceID;
|
|
@@ -135,24 +143,36 @@ public class ChunkOutputStream extends OutputStream {
|
|
|
this.streamBufferFlushSize = streamBufferFlushSize;
|
|
|
this.streamBufferMaxSize = streamBufferMaxSize;
|
|
|
this.watchTimeout = watchTimeout;
|
|
|
- this.buffer = buffer;
|
|
|
- this.ioException = null;
|
|
|
+ this.bufferList = bufferList;
|
|
|
this.checksum = checksum;
|
|
|
|
|
|
// A single thread executor handle the responses of async requests
|
|
|
responseExecutor = Executors.newSingleThreadExecutor();
|
|
|
- commitIndexList = new ArrayList<>();
|
|
|
- lastSuccessfulFlushIndex = 0;
|
|
|
+ commitIndex2flushedDataMap = new ConcurrentHashMap<>();
|
|
|
+ totalAckDataLength = 0;
|
|
|
futureList = new ArrayList<>();
|
|
|
- lastFlushPos = 0;
|
|
|
+ totalDataFlushedLength = 0;
|
|
|
+ currentBufferIndex = 0;
|
|
|
+ writtenDataLength = 0;
|
|
|
}
|
|
|
|
|
|
public BlockID getBlockID() {
|
|
|
return blockID;
|
|
|
}
|
|
|
|
|
|
- public int getLastSuccessfulFlushIndex() {
|
|
|
- return lastSuccessfulFlushIndex;
|
|
|
+ public long getTotalSuccessfulFlushedData() {
|
|
|
+ return totalAckDataLength;
|
|
|
+ }
|
|
|
+
|
|
|
+ public long getWrittenDataLength() {
|
|
|
+ return writtenDataLength;
|
|
|
+ }
|
|
|
+
|
|
|
+ private long computeBufferData() {
|
|
|
+ int dataLength =
|
|
|
+ bufferList.stream().mapToInt(Buffer::position).sum();
|
|
|
+ Preconditions.checkState(dataLength <= streamBufferMaxSize);
|
|
|
+ return dataLength;
|
|
|
}
|
|
|
|
|
|
|
|
@@ -176,139 +196,172 @@ public class ChunkOutputStream extends OutputStream {
|
|
|
if (len == 0) {
|
|
|
return;
|
|
|
}
|
|
|
- checkOpen();
|
|
|
while (len > 0) {
|
|
|
+ checkOpen();
|
|
|
int writeLen;
|
|
|
- writeLen = Math.min(chunkSize - buffer.position() % chunkSize, len);
|
|
|
- buffer.put(b, off, writeLen);
|
|
|
- if (buffer.position() % chunkSize == 0) {
|
|
|
- int pos = buffer.position() - chunkSize;
|
|
|
- int limit = buffer.position();
|
|
|
+ allocateBuffer();
|
|
|
+ ByteBuffer currentBuffer = getCurrentBuffer();
|
|
|
+ writeLen =
|
|
|
+ Math.min(chunkSize - currentBuffer.position() % chunkSize, len);
|
|
|
+ currentBuffer.put(b, off, writeLen);
|
|
|
+ if (currentBuffer.position() % chunkSize == 0) {
|
|
|
+ int pos = currentBuffer.position() - chunkSize;
|
|
|
+ int limit = currentBuffer.position();
|
|
|
writeChunk(pos, limit);
|
|
|
}
|
|
|
off += writeLen;
|
|
|
len -= writeLen;
|
|
|
- if (buffer.position() >= streamBufferFlushSize
|
|
|
- && buffer.position() % streamBufferFlushSize == 0) {
|
|
|
-
|
|
|
- lastFlushPos = buffer.position();
|
|
|
- futureList.add(handlePartialFlush());
|
|
|
+ writtenDataLength += writeLen;
|
|
|
+ if (currentBuffer.position() == streamBufferFlushSize) {
|
|
|
+ totalDataFlushedLength += streamBufferFlushSize;
|
|
|
+ handlePartialFlush();
|
|
|
}
|
|
|
- if (buffer.position() >= streamBufferMaxSize
|
|
|
- && buffer.position() % streamBufferMaxSize == 0) {
|
|
|
+ long bufferedData = computeBufferData();
|
|
|
+ // Data in the bufferList can not exceed streamBufferMaxSize
|
|
|
+ if (bufferedData == streamBufferMaxSize) {
|
|
|
handleFullBuffer();
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private ByteBuffer getCurrentBuffer() {
|
|
|
+ ByteBuffer buffer = bufferList.get(currentBufferIndex);
|
|
|
+ if (!buffer.hasRemaining()) {
|
|
|
+ currentBufferIndex =
|
|
|
+ currentBufferIndex < getMaxNumBuffers() - 1 ? ++currentBufferIndex :
|
|
|
+ 0;
|
|
|
+ }
|
|
|
+ return bufferList.get(currentBufferIndex);
|
|
|
+ }
|
|
|
+
|
|
|
+ private int getMaxNumBuffers() {
|
|
|
+ return (int)(streamBufferMaxSize/streamBufferFlushSize);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void allocateBuffer() {
|
|
|
+ for (int i = bufferList.size(); i < getMaxNumBuffers(); i++) {
|
|
|
+ bufferList.add(ByteBuffer.allocate((int)streamBufferFlushSize));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Will be called on the retryPath in case closedContainerException/
|
|
|
* TimeoutException.
|
|
|
* @param len length of data to write
|
|
|
- * @throws IOException if error occured
|
|
|
+ * @throws IOException if error occurred
|
|
|
*/
|
|
|
|
|
|
- // In this case, the data is already cached in the buffer.
|
|
|
- public void writeOnRetry(int len) throws IOException {
|
|
|
+ // In this case, the data is already cached in the currentBuffer.
|
|
|
+ public void writeOnRetry(long len) throws IOException {
|
|
|
if (len == 0) {
|
|
|
return;
|
|
|
}
|
|
|
int off = 0;
|
|
|
- checkOpen();
|
|
|
+ int pos = off;
|
|
|
while (len > 0) {
|
|
|
- int writeLen;
|
|
|
+ long writeLen;
|
|
|
writeLen = Math.min(chunkSize, len);
|
|
|
if (writeLen == chunkSize) {
|
|
|
- int pos = off;
|
|
|
int limit = pos + chunkSize;
|
|
|
writeChunk(pos, limit);
|
|
|
}
|
|
|
off += writeLen;
|
|
|
len -= writeLen;
|
|
|
+ writtenDataLength += writeLen;
|
|
|
if (off % streamBufferFlushSize == 0) {
|
|
|
- lastFlushPos = off;
|
|
|
- futureList.add(handlePartialFlush());
|
|
|
+ // reset the position to zero as now we wll readng thhe next buffer in
|
|
|
+ // the list
|
|
|
+ pos = 0;
|
|
|
+ totalDataFlushedLength += streamBufferFlushSize;
|
|
|
+ handlePartialFlush();
|
|
|
}
|
|
|
- if (off % streamBufferMaxSize == 0) {
|
|
|
+ if (computeBufferData() % streamBufferMaxSize == 0) {
|
|
|
handleFullBuffer();
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void handleResponse(
|
|
|
- ContainerProtos.ContainerCommandResponseProto response,
|
|
|
- XceiverClientAsyncReply asyncReply) {
|
|
|
- validateResponse(response);
|
|
|
- discardBuffer(asyncReply);
|
|
|
- }
|
|
|
-
|
|
|
- private void discardBuffer(XceiverClientAsyncReply asyncReply) {
|
|
|
- if (!commitIndexList.isEmpty()) {
|
|
|
- long index = commitIndexList.get(0);
|
|
|
- if (checkIfBufferDiscardRequired(asyncReply, index)) {
|
|
|
- updateFlushIndex();
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
- * just update the lastSuccessfulFlushIndex. Since we have allocated
|
|
|
- * the buffer more than the streamBufferMaxSize, we can keep on writing
|
|
|
- * to the buffer. In case of failure, we will read the data starting from
|
|
|
- * lastSuccessfulFlushIndex.
|
|
|
+ * just update the totalAckDataLength. Since we have allocated
|
|
|
+ * the currentBuffer more than the streamBufferMaxSize, we can keep on writing
|
|
|
+ * to the currentBuffer. In case of failure, we will read the data starting
|
|
|
+ * from totalAckDataLength.
|
|
|
*/
|
|
|
- private void updateFlushIndex() {
|
|
|
- lastSuccessfulFlushIndex += streamBufferFlushSize;
|
|
|
- LOG.debug("Discarding buffer till pos " + lastSuccessfulFlushIndex);
|
|
|
- if (!commitIndexList.isEmpty()) {
|
|
|
- commitIndexList.remove(0);
|
|
|
+ private void updateFlushIndex(long index) {
|
|
|
+ if (!commitIndex2flushedDataMap.isEmpty()) {
|
|
|
+ Preconditions.checkState(commitIndex2flushedDataMap.containsKey(index));
|
|
|
+ totalAckDataLength = commitIndex2flushedDataMap.remove(index);
|
|
|
+ LOG.debug("Total data successfully replicated: " + totalAckDataLength);
|
|
|
futureList.remove(0);
|
|
|
- }
|
|
|
-
|
|
|
- }
|
|
|
- /**
|
|
|
- * Check if the last commitIndex stored at the beginning of the
|
|
|
- * commitIndexList is less than equal to current commitInfo indexes.
|
|
|
- * If its true, the buffer has been successfully flushed till the
|
|
|
- * last position where flush happened.
|
|
|
- */
|
|
|
- private boolean checkIfBufferDiscardRequired(
|
|
|
- XceiverClientAsyncReply asyncReply, long commitIndex) {
|
|
|
- if (asyncReply.getCommitInfos() != null) {
|
|
|
- for (XceiverClientAsyncReply.CommitInfo info : asyncReply
|
|
|
- .getCommitInfos()) {
|
|
|
- if (info.getCommitIndex() < commitIndex) {
|
|
|
- return false;
|
|
|
- }
|
|
|
+ // Flush has been committed to required servers successful.
|
|
|
+ // just swap the bufferList head and tail after clearing.
|
|
|
+ ByteBuffer currentBuffer = bufferList.remove(0);
|
|
|
+ currentBuffer.clear();
|
|
|
+ if (currentBufferIndex != 0) {
|
|
|
+ currentBufferIndex--;
|
|
|
}
|
|
|
+ bufferList.add(currentBuffer);
|
|
|
}
|
|
|
- return true;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * This is a blocking call.It will wait for the flush till the commit index
|
|
|
- * at the head of the commitIndexList gets replicated to all or majority.
|
|
|
+ * This is a blocking call. It will wait for the flush till the commit index
|
|
|
+ * at the head of the commitIndex2flushedDataMap gets replicated to all or
|
|
|
+ * majority.
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
private void handleFullBuffer() throws IOException {
|
|
|
- if (!commitIndexList.isEmpty()) {
|
|
|
- watchForCommit(commitIndexList.get(0));
|
|
|
+ try {
|
|
|
+ checkOpen();
|
|
|
+ if (!futureList.isEmpty()) {
|
|
|
+ waitOnFlushFutures();
|
|
|
+ }
|
|
|
+ } catch (InterruptedException | ExecutionException e) {
|
|
|
+ adjustBuffersOnException();
|
|
|
+ throw new IOException(
|
|
|
+ "Unexpected Storage Container Exception: " + e.toString(), e);
|
|
|
+ }
|
|
|
+ if (!commitIndex2flushedDataMap.isEmpty()) {
|
|
|
+ watchForCommit(
|
|
|
+ commitIndex2flushedDataMap.keySet().stream().mapToLong(v -> v)
|
|
|
+ .min().getAsLong());
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private void adjustBuffers(long commitIndex) {
|
|
|
+ commitIndex2flushedDataMap.keySet().stream().forEach(index -> {
|
|
|
+ if (index <= commitIndex) {
|
|
|
+ updateFlushIndex(index);
|
|
|
+ } else {
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
+
|
|
|
+ // It may happen that once the exception is encountered , we still might
|
|
|
+ // have successfully flushed up to a certain index. Make sure the buffers
|
|
|
+ // only contain data which have not been sufficiently replicated
|
|
|
+ private void adjustBuffersOnException() {
|
|
|
+ adjustBuffers(xceiverClient.getReplicatedMinCommitIndex());
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* calls watchForCommit API of the Ratis Client. For Standalone client,
|
|
|
* it is a no op.
|
|
|
* @param commitIndex log index to watch for
|
|
|
+ * @return minimum commit index replicated to all nodes
|
|
|
* @throws IOException IOException in case watch gets timed out
|
|
|
*/
|
|
|
private void watchForCommit(long commitIndex) throws IOException {
|
|
|
checkOpen();
|
|
|
- Preconditions.checkState(!commitIndexList.isEmpty());
|
|
|
+ Preconditions.checkState(!commitIndex2flushedDataMap.isEmpty());
|
|
|
try {
|
|
|
- xceiverClient.watchForCommit(commitIndex, watchTimeout);
|
|
|
+ long index =
|
|
|
+ xceiverClient.watchForCommit(commitIndex, watchTimeout);
|
|
|
+ adjustBuffers(index);
|
|
|
} catch (TimeoutException | InterruptedException | ExecutionException e) {
|
|
|
LOG.warn("watchForCommit failed for index " + commitIndex, e);
|
|
|
+ adjustBuffersOnException();
|
|
|
throw new IOException(
|
|
|
"Unexpected Storage Container Exception: " + e.toString(), e);
|
|
|
}
|
|
@@ -317,68 +370,79 @@ public class ChunkOutputStream extends OutputStream {
|
|
|
private CompletableFuture<ContainerProtos.
|
|
|
ContainerCommandResponseProto> handlePartialFlush()
|
|
|
throws IOException {
|
|
|
+ checkOpen();
|
|
|
+ long flushPos = totalDataFlushedLength;
|
|
|
String requestId =
|
|
|
traceID + ContainerProtos.Type.PutBlock + chunkIndex + blockID;
|
|
|
+ CompletableFuture<ContainerProtos.
|
|
|
+ ContainerCommandResponseProto> flushFuture;
|
|
|
try {
|
|
|
XceiverClientAsyncReply asyncReply =
|
|
|
putBlockAsync(xceiverClient, containerBlockData.build(), requestId);
|
|
|
CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
|
|
|
asyncReply.getResponse();
|
|
|
-
|
|
|
- return future.thenApplyAsync(e -> {
|
|
|
- handleResponse(e, asyncReply);
|
|
|
+ flushFuture = future.thenApplyAsync(e -> {
|
|
|
+ try {
|
|
|
+ validateResponse(e);
|
|
|
+ } catch (IOException sce) {
|
|
|
+ future.completeExceptionally(sce);
|
|
|
+ return e;
|
|
|
+ }
|
|
|
// if the ioException is not set, putBlock is successful
|
|
|
if (ioException == null) {
|
|
|
LOG.debug(
|
|
|
- "Adding index " + asyncReply.getLogIndex() + " commitList size "
|
|
|
- + commitIndexList.size());
|
|
|
+ "Adding index " + asyncReply.getLogIndex() + " commitMap size "
|
|
|
+ + commitIndex2flushedDataMap.size());
|
|
|
BlockID responseBlockID = BlockID.getFromProtobuf(
|
|
|
e.getPutBlock().getCommittedBlockLength().getBlockID());
|
|
|
Preconditions.checkState(blockID.getContainerBlockID()
|
|
|
.equals(responseBlockID.getContainerBlockID()));
|
|
|
// updates the bcsId of the block
|
|
|
blockID = responseBlockID;
|
|
|
- long index = asyncReply.getLogIndex();
|
|
|
// for standalone protocol, logIndex will always be 0.
|
|
|
- if (index != 0) {
|
|
|
- commitIndexList.add(index);
|
|
|
- } else {
|
|
|
- updateFlushIndex();
|
|
|
- }
|
|
|
+ commitIndex2flushedDataMap.put(asyncReply.getLogIndex(), flushPos);
|
|
|
}
|
|
|
return e;
|
|
|
- }, responseExecutor);
|
|
|
+ }, responseExecutor).exceptionally(e -> {
|
|
|
+ LOG.debug(
|
|
|
+ "putBlock failed for blockID " + blockID + " with exception " + e
|
|
|
+ .getLocalizedMessage());
|
|
|
+ CompletionException ce = new CompletionException(e);
|
|
|
+ setIoException(ce);
|
|
|
+ throw ce;
|
|
|
+ });
|
|
|
} catch (IOException | InterruptedException | ExecutionException e) {
|
|
|
throw new IOException(
|
|
|
"Unexpected Storage Container Exception: " + e.toString(), e);
|
|
|
}
|
|
|
+ futureList.add(flushFuture);
|
|
|
+ return flushFuture;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void flush() throws IOException {
|
|
|
if (xceiverClientManager != null && xceiverClient != null
|
|
|
- && buffer != null) {
|
|
|
+ && bufferList != null) {
|
|
|
checkOpen();
|
|
|
- if (buffer.position() > 0 && lastSuccessfulFlushIndex != buffer
|
|
|
- .position()) {
|
|
|
+ int bufferSize = bufferList.size();
|
|
|
+ if (bufferSize > 0) {
|
|
|
try {
|
|
|
-
|
|
|
- // flush the last chunk data residing on the buffer
|
|
|
- if (buffer.position() % chunkSize > 0) {
|
|
|
- int pos = buffer.position() - (buffer.position() % chunkSize);
|
|
|
- writeChunk(pos, buffer.position());
|
|
|
- }
|
|
|
- if (lastFlushPos != buffer.position()) {
|
|
|
- lastFlushPos = buffer.position();
|
|
|
+ // flush the last chunk data residing on the currentBuffer
|
|
|
+ if (totalDataFlushedLength < writtenDataLength) {
|
|
|
+ ByteBuffer currentBuffer = getCurrentBuffer();
|
|
|
+ int pos = currentBuffer.position() - (currentBuffer.position()
|
|
|
+ % chunkSize);
|
|
|
+ int limit = currentBuffer.position() - pos;
|
|
|
+ writeChunk(pos, currentBuffer.position());
|
|
|
+ totalDataFlushedLength += limit;
|
|
|
handlePartialFlush();
|
|
|
}
|
|
|
- CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(
|
|
|
- futureList.toArray(new CompletableFuture[futureList.size()]));
|
|
|
- combinedFuture.get();
|
|
|
+ waitOnFlushFutures();
|
|
|
// just check again if the exception is hit while waiting for the
|
|
|
// futures to ensure flush has indeed succeeded
|
|
|
checkOpen();
|
|
|
} catch (InterruptedException | ExecutionException e) {
|
|
|
+ adjustBuffersOnException();
|
|
|
throw new IOException(
|
|
|
"Unexpected Storage Container Exception: " + e.toString(), e);
|
|
|
}
|
|
@@ -388,11 +452,11 @@ public class ChunkOutputStream extends OutputStream {
|
|
|
|
|
|
private void writeChunk(int pos, int limit) throws IOException {
|
|
|
// Please note : We are not flipping the slice when we write since
|
|
|
- // the slices are pointing the buffer start and end as needed for
|
|
|
+ // the slices are pointing the currentBuffer start and end as needed for
|
|
|
// the chunk write. Also please note, Duplicate does not create a
|
|
|
// copy of data, it only creates metadata that points to the data
|
|
|
// stream.
|
|
|
- ByteBuffer chunk = buffer.duplicate();
|
|
|
+ ByteBuffer chunk = bufferList.get(currentBufferIndex).duplicate();
|
|
|
chunk.position(pos);
|
|
|
chunk.limit(limit);
|
|
|
writeChunkToContainer(chunk);
|
|
@@ -401,49 +465,78 @@ public class ChunkOutputStream extends OutputStream {
|
|
|
@Override
|
|
|
public void close() throws IOException {
|
|
|
if (xceiverClientManager != null && xceiverClient != null
|
|
|
- && buffer != null) {
|
|
|
- try {
|
|
|
- if (buffer.position() > lastFlushPos) {
|
|
|
- int pos = buffer.position() - (buffer.position() % chunkSize);
|
|
|
- writeChunk(pos, buffer.position());
|
|
|
- futureList.add(handlePartialFlush());
|
|
|
- }
|
|
|
- CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(
|
|
|
- futureList.toArray(new CompletableFuture[futureList.size()]));
|
|
|
-
|
|
|
- // wait for all the transactions to complete
|
|
|
- combinedFuture.get();
|
|
|
-
|
|
|
- // irrespective of whether the commitIndexList is empty or not,
|
|
|
- // ensure there is no exception set(For Standalone Protocol)
|
|
|
- checkOpen();
|
|
|
- if (!commitIndexList.isEmpty()) {
|
|
|
- // wait for the last commit index in the commitIndexList to get
|
|
|
- // committed to all or majority of nodes in case timeout happens.
|
|
|
- long lastIndex = commitIndexList.get(commitIndexList.size() - 1);
|
|
|
- LOG.debug(
|
|
|
- "waiting for last flush Index " + lastIndex + " to catch up");
|
|
|
- watchForCommit(lastIndex);
|
|
|
- updateFlushIndex();
|
|
|
+ && bufferList != null) {
|
|
|
+ int bufferSize = bufferList.size();
|
|
|
+ if (bufferSize > 0) {
|
|
|
+ try {
|
|
|
+ // flush the last chunk data residing on the currentBuffer
|
|
|
+ if (totalDataFlushedLength < writtenDataLength) {
|
|
|
+ ByteBuffer currentBuffer = getCurrentBuffer();
|
|
|
+ int pos = currentBuffer.position() - (currentBuffer.position()
|
|
|
+ % chunkSize);
|
|
|
+ int limit = currentBuffer.position() - pos;
|
|
|
+ writeChunk(pos, currentBuffer.position());
|
|
|
+ totalDataFlushedLength += limit;
|
|
|
+ handlePartialFlush();
|
|
|
+ }
|
|
|
+ waitOnFlushFutures();
|
|
|
+ // irrespective of whether the commitIndex2flushedDataMap is empty
|
|
|
+ // or not, ensure there is no exception set
|
|
|
+ checkOpen();
|
|
|
+ if (!commitIndex2flushedDataMap.isEmpty()) {
|
|
|
+ // wait for the last commit index in the commitIndex2flushedDataMap
|
|
|
+ // to get committed to all or majority of nodes in case timeout
|
|
|
+ // happens.
|
|
|
+ long lastIndex =
|
|
|
+ commitIndex2flushedDataMap.keySet().stream()
|
|
|
+ .mapToLong(v -> v).max().getAsLong();
|
|
|
+ LOG.debug(
|
|
|
+ "waiting for last flush Index " + lastIndex + " to catch up");
|
|
|
+ watchForCommit(lastIndex);
|
|
|
+ }
|
|
|
+ } catch (InterruptedException | ExecutionException e) {
|
|
|
+ adjustBuffersOnException();
|
|
|
+ throw new IOException(
|
|
|
+ "Unexpected Storage Container Exception: " + e.toString(), e);
|
|
|
+ } finally {
|
|
|
+ cleanup();
|
|
|
}
|
|
|
- } catch (InterruptedException | ExecutionException e) {
|
|
|
- throw new IOException(
|
|
|
- "Unexpected Storage Container Exception: " + e.toString(), e);
|
|
|
- } finally {
|
|
|
- cleanup();
|
|
|
}
|
|
|
+ // clear the currentBuffer
|
|
|
+ bufferList.stream().forEach(ByteBuffer::clear);
|
|
|
}
|
|
|
- // clear the buffer
|
|
|
- buffer.clear();
|
|
|
+ }
|
|
|
+
|
|
|
+ private void waitOnFlushFutures()
|
|
|
+ throws InterruptedException, ExecutionException {
|
|
|
+ CompletableFuture<Void> combinedFuture = CompletableFuture
|
|
|
+ .allOf(futureList.toArray(new CompletableFuture[futureList.size()]));
|
|
|
+ // wait for all the transactions to complete
|
|
|
+ combinedFuture.get();
|
|
|
}
|
|
|
|
|
|
private void validateResponse(
|
|
|
- ContainerProtos.ContainerCommandResponseProto responseProto) {
|
|
|
+ ContainerProtos.ContainerCommandResponseProto responseProto)
|
|
|
+ throws IOException {
|
|
|
try {
|
|
|
+ // if the ioException is already set, it means a prev request has failed
|
|
|
+ // just throw the exception. The current operation will fail with the
|
|
|
+ // original error
|
|
|
+ if (ioException != null) {
|
|
|
+ throw ioException;
|
|
|
+ }
|
|
|
ContainerProtocolCalls.validateContainerResponse(responseProto);
|
|
|
} catch (StorageContainerException sce) {
|
|
|
- ioException = new IOException(
|
|
|
- "Unexpected Storage Container Exception: " + sce.toString(), sce);
|
|
|
+ LOG.error("Unexpected Storage Container Exception: ", sce);
|
|
|
+ setIoException(sce);
|
|
|
+ throw sce;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void setIoException(Exception e) {
|
|
|
+ if (ioException != null) {
|
|
|
+ ioException = new IOException(
|
|
|
+ "Unexpected Storage Container Exception: " + e.toString(), e);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -457,7 +550,10 @@ public class ChunkOutputStream extends OutputStream {
|
|
|
futureList.clear();
|
|
|
}
|
|
|
futureList = null;
|
|
|
- commitIndexList = null;
|
|
|
+ if (commitIndex2flushedDataMap != null) {
|
|
|
+ commitIndex2flushedDataMap.clear();
|
|
|
+ }
|
|
|
+ commitIndex2flushedDataMap = null;
|
|
|
responseExecutor.shutdown();
|
|
|
}
|
|
|
|
|
@@ -471,6 +567,7 @@ public class ChunkOutputStream extends OutputStream {
|
|
|
if (xceiverClient == null) {
|
|
|
throw new IOException("ChunkOutputStream has been closed.");
|
|
|
} else if (ioException != null) {
|
|
|
+ adjustBuffersOnException();
|
|
|
throw ioException;
|
|
|
}
|
|
|
}
|
|
@@ -504,16 +601,27 @@ public class ChunkOutputStream extends OutputStream {
|
|
|
CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
|
|
|
asyncReply.getResponse();
|
|
|
future.thenApplyAsync(e -> {
|
|
|
- handleResponse(e, asyncReply);
|
|
|
+ try {
|
|
|
+ validateResponse(e);
|
|
|
+ } catch (IOException sce) {
|
|
|
+ future.completeExceptionally(sce);
|
|
|
+ }
|
|
|
return e;
|
|
|
- }, responseExecutor);
|
|
|
+ }, responseExecutor).exceptionally(e -> {
|
|
|
+ LOG.debug(
|
|
|
+ "writing chunk failed " + chunkInfo.getChunkName() + " blockID "
|
|
|
+ + blockID + " with exception " + e.getLocalizedMessage());
|
|
|
+ CompletionException ce = new CompletionException(e);
|
|
|
+ setIoException(ce);
|
|
|
+ throw ce;
|
|
|
+ });
|
|
|
} catch (IOException | InterruptedException | ExecutionException e) {
|
|
|
throw new IOException(
|
|
|
"Unexpected Storage Container Exception: " + e.toString(), e);
|
|
|
}
|
|
|
LOG.debug(
|
|
|
"writing chunk " + chunkInfo.getChunkName() + " blockID " + blockID
|
|
|
- + " length " + chunk.remaining());
|
|
|
+ + " length " + effectiveChunkSize);
|
|
|
containerBlockData.addChunks(chunkInfo);
|
|
|
}
|
|
|
}
|