|
@@ -48,16 +48,12 @@ import java.util.UUID;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
-import java.util.concurrent.ConcurrentSkipListMap;
|
|
|
|
-import java.util.concurrent.ConcurrentHashMap;
|
|
|
|
import java.util.concurrent.ExecutorService;
|
|
import java.util.concurrent.ExecutorService;
|
|
import java.util.concurrent.CompletableFuture;
|
|
import java.util.concurrent.CompletableFuture;
|
|
-import java.util.concurrent.TimeoutException;
|
|
|
|
import java.util.concurrent.CompletionException;
|
|
import java.util.concurrent.CompletionException;
|
|
import java.util.concurrent.ExecutionException;
|
|
import java.util.concurrent.ExecutionException;
|
|
import java.util.concurrent.Executors;
|
|
import java.util.concurrent.Executors;
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
-import java.util.stream.Collectors;
|
|
|
|
|
|
|
|
import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls
|
|
import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls
|
|
.putBlockAsync;
|
|
.putBlockAsync;
|
|
@@ -97,7 +93,6 @@ public class BlockOutputStream extends OutputStream {
|
|
private int chunkSize;
|
|
private int chunkSize;
|
|
private final long streamBufferFlushSize;
|
|
private final long streamBufferFlushSize;
|
|
private final long streamBufferMaxSize;
|
|
private final long streamBufferMaxSize;
|
|
- private final long watchTimeout;
|
|
|
|
private BufferPool bufferPool;
|
|
private BufferPool bufferPool;
|
|
// The IOException will be set by response handling thread in case there is an
|
|
// 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
|
|
// exception received in the response. If the exception is set, the next
|
|
@@ -111,10 +106,6 @@ public class BlockOutputStream extends OutputStream {
|
|
// effective data write attempted so far for the block
|
|
// effective data write attempted so far for the block
|
|
private long writtenDataLength;
|
|
private long writtenDataLength;
|
|
|
|
|
|
- // total data which has been successfully flushed and acknowledged
|
|
|
|
- // by all servers
|
|
|
|
- private long totalAckDataLength;
|
|
|
|
-
|
|
|
|
// List containing buffers for which the putBlock call will
|
|
// List containing buffers for which the putBlock call will
|
|
// update the length in the datanodes. This list will just maintain
|
|
// update the length in the datanodes. This list will just maintain
|
|
// references to the buffers in the BufferPool which will be cleared
|
|
// references to the buffers in the BufferPool which will be cleared
|
|
@@ -123,17 +114,10 @@ public class BlockOutputStream extends OutputStream {
|
|
// which got written between successive putBlock calls.
|
|
// which got written between successive putBlock calls.
|
|
private List<ByteBuffer> bufferList;
|
|
private List<ByteBuffer> bufferList;
|
|
|
|
|
|
- // future Map to hold up all putBlock futures
|
|
|
|
- private ConcurrentHashMap<Long,
|
|
|
|
- CompletableFuture<ContainerProtos.ContainerCommandResponseProto>>
|
|
|
|
- futureMap;
|
|
|
|
-
|
|
|
|
- // The map should maintain the keys (logIndexes) in order so that while
|
|
|
|
- // removing we always end up updating incremented data flushed length.
|
|
|
|
|
|
+ // This object will maintain the commitIndexes and byteBufferList in order
|
|
// Also, corresponding to the logIndex, the corresponding list of buffers will
|
|
// Also, corresponding to the logIndex, the corresponding list of buffers will
|
|
// be released from the buffer pool.
|
|
// be released from the buffer pool.
|
|
- private ConcurrentSkipListMap<Long, List<ByteBuffer>>
|
|
|
|
- commitIndex2flushedDataMap;
|
|
|
|
|
|
+ private final CommitWatcher commitWatcher;
|
|
|
|
|
|
private List<DatanodeDetails> failedServers;
|
|
private List<DatanodeDetails> failedServers;
|
|
|
|
|
|
@@ -175,20 +159,17 @@ public class BlockOutputStream extends OutputStream {
|
|
this.chunkIndex = 0;
|
|
this.chunkIndex = 0;
|
|
this.streamBufferFlushSize = streamBufferFlushSize;
|
|
this.streamBufferFlushSize = streamBufferFlushSize;
|
|
this.streamBufferMaxSize = streamBufferMaxSize;
|
|
this.streamBufferMaxSize = streamBufferMaxSize;
|
|
- this.watchTimeout = watchTimeout;
|
|
|
|
this.bufferPool = bufferPool;
|
|
this.bufferPool = bufferPool;
|
|
this.checksumType = checksumType;
|
|
this.checksumType = checksumType;
|
|
this.bytesPerChecksum = bytesPerChecksum;
|
|
this.bytesPerChecksum = bytesPerChecksum;
|
|
|
|
|
|
// A single thread executor handle the responses of async requests
|
|
// A single thread executor handle the responses of async requests
|
|
responseExecutor = Executors.newSingleThreadExecutor();
|
|
responseExecutor = Executors.newSingleThreadExecutor();
|
|
- commitIndex2flushedDataMap = new ConcurrentSkipListMap<>();
|
|
|
|
- totalAckDataLength = 0;
|
|
|
|
- futureMap = new ConcurrentHashMap<>();
|
|
|
|
|
|
+ commitWatcher = new CommitWatcher(bufferPool, xceiverClient, watchTimeout);
|
|
|
|
+ bufferList = null;
|
|
totalDataFlushedLength = 0;
|
|
totalDataFlushedLength = 0;
|
|
writtenDataLength = 0;
|
|
writtenDataLength = 0;
|
|
failedServers = Collections.emptyList();
|
|
failedServers = Collections.emptyList();
|
|
- bufferList = null;
|
|
|
|
ioException = new AtomicReference<>(null);
|
|
ioException = new AtomicReference<>(null);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -198,7 +179,7 @@ public class BlockOutputStream extends OutputStream {
|
|
}
|
|
}
|
|
|
|
|
|
public long getTotalAckDataLength() {
|
|
public long getTotalAckDataLength() {
|
|
- return totalAckDataLength;
|
|
|
|
|
|
+ return commitWatcher.getTotalAckDataLength();
|
|
}
|
|
}
|
|
|
|
|
|
public long getWrittenDataLength() {
|
|
public long getWrittenDataLength() {
|
|
@@ -230,7 +211,7 @@ public class BlockOutputStream extends OutputStream {
|
|
|
|
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
public Map<Long, List<ByteBuffer>> getCommitIndex2flushedDataMap() {
|
|
public Map<Long, List<ByteBuffer>> getCommitIndex2flushedDataMap() {
|
|
- return commitIndex2flushedDataMap;
|
|
|
|
|
|
+ return commitWatcher.getCommitIndex2flushedDataMap();
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -333,34 +314,6 @@ public class BlockOutputStream extends OutputStream {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * just update the totalAckDataLength. In case of failure,
|
|
|
|
- * we will read the data starting from totalAckDataLength.
|
|
|
|
- */
|
|
|
|
- private void updateFlushIndex(List<Long> indexes) {
|
|
|
|
- Preconditions.checkArgument(!commitIndex2flushedDataMap.isEmpty());
|
|
|
|
- for (long index : indexes) {
|
|
|
|
- Preconditions.checkState(commitIndex2flushedDataMap.containsKey(index));
|
|
|
|
- List<ByteBuffer> buffers = commitIndex2flushedDataMap.remove(index);
|
|
|
|
- long length = buffers.stream().mapToLong(value -> {
|
|
|
|
- int pos = value.position();
|
|
|
|
- Preconditions.checkArgument(pos <= chunkSize);
|
|
|
|
- return pos;
|
|
|
|
- }).sum();
|
|
|
|
- // totalAckDataLength replicated yet should always be incremented
|
|
|
|
- // with the current length being returned from commitIndex2flushedDataMap.
|
|
|
|
- totalAckDataLength += length;
|
|
|
|
- LOG.debug("Total data successfully replicated: " + totalAckDataLength);
|
|
|
|
- futureMap.remove(totalAckDataLength);
|
|
|
|
- // Flush has been committed to required servers successful.
|
|
|
|
- // just release the current buffer from the buffer pool corresponding
|
|
|
|
- // to the buffers that have been committed with the putBlock call.
|
|
|
|
- for (ByteBuffer byteBuffer : buffers) {
|
|
|
|
- bufferPool.releaseBuffer(byteBuffer);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* This is a blocking call. It will wait for the flush till the commit index
|
|
* 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
|
|
* at the head of the commitIndex2flushedDataMap gets replicated to all or
|
|
@@ -370,7 +323,7 @@ public class BlockOutputStream extends OutputStream {
|
|
private void handleFullBuffer() throws IOException {
|
|
private void handleFullBuffer() throws IOException {
|
|
try {
|
|
try {
|
|
checkOpen();
|
|
checkOpen();
|
|
- if (!futureMap.isEmpty()) {
|
|
|
|
|
|
+ if (!commitWatcher.getFutureMap().isEmpty()) {
|
|
waitOnFlushFutures();
|
|
waitOnFlushFutures();
|
|
}
|
|
}
|
|
} catch (InterruptedException | ExecutionException e) {
|
|
} catch (InterruptedException | ExecutionException e) {
|
|
@@ -378,47 +331,31 @@ public class BlockOutputStream extends OutputStream {
|
|
adjustBuffersOnException();
|
|
adjustBuffersOnException();
|
|
throw getIoException();
|
|
throw getIoException();
|
|
}
|
|
}
|
|
- if (!commitIndex2flushedDataMap.isEmpty()) {
|
|
|
|
- watchForCommit(
|
|
|
|
- commitIndex2flushedDataMap.keySet().stream().mapToLong(v -> v)
|
|
|
|
- .min().getAsLong());
|
|
|
|
- }
|
|
|
|
|
|
+ watchForCommit(true);
|
|
}
|
|
}
|
|
|
|
|
|
- private void adjustBuffers(long commitIndex) {
|
|
|
|
- List<Long> keyList = commitIndex2flushedDataMap.keySet().stream()
|
|
|
|
- .filter(p -> p <= commitIndex).collect(Collectors.toList());
|
|
|
|
- if (keyList.isEmpty()) {
|
|
|
|
- return;
|
|
|
|
- } else {
|
|
|
|
- updateFlushIndex(keyList);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
|
|
// It may happen that once the exception is encountered , we still might
|
|
// It may happen that once the exception is encountered , we still might
|
|
// have successfully flushed up to a certain index. Make sure the buffers
|
|
// have successfully flushed up to a certain index. Make sure the buffers
|
|
// only contain data which have not been sufficiently replicated
|
|
// only contain data which have not been sufficiently replicated
|
|
private void adjustBuffersOnException() {
|
|
private void adjustBuffersOnException() {
|
|
- adjustBuffers(xceiverClient.getReplicatedMinCommitIndex());
|
|
|
|
|
|
+ commitWatcher.releaseBuffersOnException();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
* calls watchForCommit API of the Ratis Client. For Standalone client,
|
|
* calls watchForCommit API of the Ratis Client. For Standalone client,
|
|
* it is a no op.
|
|
* it is a no op.
|
|
- * @param commitIndex log index to watch for
|
|
|
|
|
|
+ * @param bufferFull flag indicating whether bufferFull condition is hit or
|
|
|
|
+ * its called as part flush/close
|
|
* @return minimum commit index replicated to all nodes
|
|
* @return minimum commit index replicated to all nodes
|
|
* @throws IOException IOException in case watch gets timed out
|
|
* @throws IOException IOException in case watch gets timed out
|
|
*/
|
|
*/
|
|
- private void watchForCommit(long commitIndex) throws IOException {
|
|
|
|
|
|
+ private void watchForCommit(boolean bufferFull) throws IOException {
|
|
checkOpen();
|
|
checkOpen();
|
|
- Preconditions.checkState(!commitIndex2flushedDataMap.isEmpty());
|
|
|
|
- long index;
|
|
|
|
try {
|
|
try {
|
|
- XceiverClientReply reply =
|
|
|
|
- xceiverClient.watchForCommit(commitIndex, watchTimeout);
|
|
|
|
- if (reply == null) {
|
|
|
|
- index = 0;
|
|
|
|
- } else {
|
|
|
|
|
|
+ XceiverClientReply reply = bufferFull ?
|
|
|
|
+ commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
|
|
|
|
+ if (reply != null) {
|
|
List<DatanodeDetails> dnList = reply.getDatanodes();
|
|
List<DatanodeDetails> dnList = reply.getDatanodes();
|
|
if (!dnList.isEmpty()) {
|
|
if (!dnList.isEmpty()) {
|
|
if (failedServers.isEmpty()) {
|
|
if (failedServers.isEmpty()) {
|
|
@@ -426,13 +363,9 @@ public class BlockOutputStream extends OutputStream {
|
|
}
|
|
}
|
|
failedServers.addAll(dnList);
|
|
failedServers.addAll(dnList);
|
|
}
|
|
}
|
|
- index = reply.getLogIndex();
|
|
|
|
}
|
|
}
|
|
- adjustBuffers(index);
|
|
|
|
- } catch (TimeoutException | InterruptedException | ExecutionException e) {
|
|
|
|
- LOG.warn("watchForCommit failed for index " + commitIndex, e);
|
|
|
|
- setIoException(e);
|
|
|
|
- adjustBuffersOnException();
|
|
|
|
|
|
+ } catch (IOException ioe) {
|
|
|
|
+ setIoException(ioe);
|
|
throw getIoException();
|
|
throw getIoException();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -471,14 +404,14 @@ public class BlockOutputStream extends OutputStream {
|
|
blockID = responseBlockID;
|
|
blockID = responseBlockID;
|
|
LOG.debug(
|
|
LOG.debug(
|
|
"Adding index " + asyncReply.getLogIndex() + " commitMap size "
|
|
"Adding index " + asyncReply.getLogIndex() + " commitMap size "
|
|
- + commitIndex2flushedDataMap.size() + " flushLength "
|
|
|
|
|
|
+ + commitWatcher.getCommitInfoMapSize() + " flushLength "
|
|
+ flushPos + " numBuffers " + byteBufferList.size()
|
|
+ flushPos + " numBuffers " + byteBufferList.size()
|
|
+ " blockID " + blockID + " bufferPool size" + bufferPool
|
|
+ " blockID " + blockID + " bufferPool size" + bufferPool
|
|
.getSize() + " currentBufferIndex " + bufferPool
|
|
.getSize() + " currentBufferIndex " + bufferPool
|
|
.getCurrentBufferIndex());
|
|
.getCurrentBufferIndex());
|
|
// for standalone protocol, logIndex will always be 0.
|
|
// for standalone protocol, logIndex will always be 0.
|
|
- commitIndex2flushedDataMap
|
|
|
|
- .put(asyncReply.getLogIndex(), byteBufferList);
|
|
|
|
|
|
+ commitWatcher
|
|
|
|
+ .updateCommitInfoMap(asyncReply.getLogIndex(), byteBufferList);
|
|
}
|
|
}
|
|
return e;
|
|
return e;
|
|
}, responseExecutor).exceptionally(e -> {
|
|
}, responseExecutor).exceptionally(e -> {
|
|
@@ -493,7 +426,7 @@ public class BlockOutputStream extends OutputStream {
|
|
throw new IOException(
|
|
throw new IOException(
|
|
"Unexpected Storage Container Exception: " + e.toString(), e);
|
|
"Unexpected Storage Container Exception: " + e.toString(), e);
|
|
}
|
|
}
|
|
- futureMap.put(flushPos, flushFuture);
|
|
|
|
|
|
+ commitWatcher.getFutureMap().put(flushPos, flushFuture);
|
|
return flushFuture;
|
|
return flushFuture;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -553,18 +486,7 @@ public class BlockOutputStream extends OutputStream {
|
|
executePutBlock();
|
|
executePutBlock();
|
|
}
|
|
}
|
|
waitOnFlushFutures();
|
|
waitOnFlushFutures();
|
|
- 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);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
|
|
+ watchForCommit(false);
|
|
// just check again if the exception is hit while waiting for the
|
|
// just check again if the exception is hit while waiting for the
|
|
// futures to ensure flush has indeed succeeded
|
|
// futures to ensure flush has indeed succeeded
|
|
|
|
|
|
@@ -594,11 +516,11 @@ public class BlockOutputStream extends OutputStream {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
-
|
|
|
|
private void waitOnFlushFutures()
|
|
private void waitOnFlushFutures()
|
|
throws InterruptedException, ExecutionException {
|
|
throws InterruptedException, ExecutionException {
|
|
CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(
|
|
CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(
|
|
- futureMap.values().toArray(new CompletableFuture[futureMap.size()]));
|
|
|
|
|
|
+ commitWatcher.getFutureMap().values().toArray(
|
|
|
|
+ new CompletableFuture[commitWatcher.getFutureMap().size()]));
|
|
// wait for all the transactions to complete
|
|
// wait for all the transactions to complete
|
|
combinedFuture.get();
|
|
combinedFuture.get();
|
|
}
|
|
}
|
|
@@ -637,18 +559,11 @@ public class BlockOutputStream extends OutputStream {
|
|
}
|
|
}
|
|
xceiverClientManager = null;
|
|
xceiverClientManager = null;
|
|
xceiverClient = null;
|
|
xceiverClient = null;
|
|
- if (futureMap != null) {
|
|
|
|
- futureMap.clear();
|
|
|
|
- }
|
|
|
|
- futureMap = null;
|
|
|
|
|
|
+ commitWatcher.cleanup();
|
|
if (bufferList != null) {
|
|
if (bufferList != null) {
|
|
bufferList.clear();
|
|
bufferList.clear();
|
|
}
|
|
}
|
|
bufferList = null;
|
|
bufferList = null;
|
|
- if (commitIndex2flushedDataMap != null) {
|
|
|
|
- commitIndex2flushedDataMap.clear();
|
|
|
|
- }
|
|
|
|
- commitIndex2flushedDataMap = null;
|
|
|
|
responseExecutor.shutdown();
|
|
responseExecutor.shutdown();
|
|
}
|
|
}
|
|
|
|
|