-
Notifications
You must be signed in to change notification settings - Fork 588
HDDS-12469. mark statemachine unhealthy for write operation timeout #8022
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 9 commits
a138456
93b9dc4
75bb186
d6a6a0f
71191ba
bf48067
b9dbb66
3b07686
2b37bbe
29da870
c743146
c5afb02
b8eae7e
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -35,17 +35,22 @@ | |
| import java.util.Iterator; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.NavigableMap; | ||
| import java.util.Objects; | ||
| import java.util.Optional; | ||
| import java.util.Set; | ||
| import java.util.SortedMap; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.concurrent.CompletionException; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.ConcurrentSkipListMap; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.Executors; | ||
| import java.util.concurrent.Future; | ||
| import java.util.concurrent.Semaphore; | ||
| import java.util.concurrent.ThreadFactory; | ||
| import java.util.concurrent.ThreadPoolExecutor; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import java.util.function.Consumer; | ||
| import org.apache.hadoop.hdds.HddsUtils; | ||
|
|
@@ -187,13 +192,38 @@ long getStartTime() { | |
| } | ||
| } | ||
|
|
||
| static class WriteFutures { | ||
| private final Future<ContainerCommandResponseProto> writeChunkFuture; | ||
| private final CompletableFuture<Message> raftFuture; | ||
| private final long startTime; | ||
sumitagrawl marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| WriteFutures(Future<ContainerCommandResponseProto> writeChunkFuture, | ||
| CompletableFuture<Message> raftFuture, long startTime) { | ||
| this.writeChunkFuture = writeChunkFuture; | ||
| this.raftFuture = raftFuture; | ||
| this.startTime = startTime; | ||
| } | ||
|
|
||
| public Future<ContainerCommandResponseProto> getWriteChunkFuture() { | ||
| return writeChunkFuture; | ||
| } | ||
|
|
||
| public CompletableFuture<Message> getRaftFuture() { | ||
| return raftFuture; | ||
| } | ||
|
|
||
| long getStartTime() { | ||
| return startTime; | ||
| } | ||
| } | ||
|
|
||
| private final SimpleStateMachineStorage storage = | ||
| new SimpleStateMachineStorage(); | ||
| private final ContainerDispatcher dispatcher; | ||
| private final ContainerController containerController; | ||
| private final XceiverServerRatis ratisServer; | ||
| private final ConcurrentHashMap<Long, | ||
| CompletableFuture<ContainerCommandResponseProto>> writeChunkFutureMap; | ||
| private final NavigableMap<Long, WriteFutures> writeChunkFutureMap; | ||
| private final long writeChunkWaitMaxNs; | ||
|
|
||
| // keeps track of the containers created per pipeline | ||
| private final Map<Long, Long> container2BCSIDMap; | ||
|
|
@@ -229,7 +259,7 @@ public ContainerStateMachine(HddsDatanodeService hddsDatanodeService, RaftGroupI | |
| this.containerController = containerController; | ||
| this.ratisServer = ratisServer; | ||
| metrics = CSMMetrics.create(gid); | ||
| this.writeChunkFutureMap = new ConcurrentHashMap<>(); | ||
| this.writeChunkFutureMap = new ConcurrentSkipListMap<>(); | ||
| applyTransactionCompletionMap = new ConcurrentHashMap<>(); | ||
| this.unhealthyContainers = ConcurrentHashMap.newKeySet(); | ||
| long pendingRequestsBytesLimit = (long)conf.getStorageSize( | ||
|
|
@@ -273,6 +303,8 @@ public ContainerStateMachine(HddsDatanodeService hddsDatanodeService, RaftGroupI | |
| this.waitOnBothFollowers = conf.getObject( | ||
| DatanodeConfiguration.class).waitOnAllFollowers(); | ||
|
|
||
| this.writeChunkWaitMaxNs = conf.getTimeDuration(ScmConfigKeys.HDDS_CONTAINER_RATIS_STATEMACHINE_WRITE_WAIT_INTERVAL, | ||
| ScmConfigKeys.HDDS_CONTAINER_RATIS_STATEMACHINE_WRITE_WAIT_INTERVAL_NS_DEFAULT, TimeUnit.NANOSECONDS); | ||
| } | ||
|
|
||
| private void validatePeers() throws IOException { | ||
|
|
@@ -542,6 +574,16 @@ private ContainerCommandResponseProto dispatchCommand( | |
| private CompletableFuture<Message> writeStateMachineData( | ||
| ContainerCommandRequestProto requestProto, long entryIndex, long term, | ||
| long startTime) { | ||
| final WriteFutures previous = writeChunkFutureMap.get(entryIndex); | ||
| if (previous != null) { | ||
| // generally state machine will wait forever, for precaution, a check is added if retry happens. | ||
| return previous.getRaftFuture(); | ||
| } | ||
| try { | ||
| validateLongRunningWrite(); | ||
| } catch (StorageContainerException e) { | ||
| return completeExceptionally(e); | ||
| } | ||
| final WriteChunkRequestProto write = requestProto.getWriteChunk(); | ||
| RaftServer server = ratisServer.getServer(); | ||
| Preconditions.checkArgument(!write.getData().isEmpty()); | ||
|
|
@@ -564,22 +606,25 @@ private CompletableFuture<Message> writeStateMachineData( | |
| .setContainer2BCSIDMap(container2BCSIDMap) | ||
| .build(); | ||
| CompletableFuture<Message> raftFuture = new CompletableFuture<>(); | ||
| // ensure the write chunk happens asynchronously in writeChunkExecutor pool | ||
| // thread. | ||
| CompletableFuture<ContainerCommandResponseProto> writeChunkFuture = | ||
| CompletableFuture.supplyAsync(() -> { | ||
| // ensure the write chunk happens asynchronously in writeChunkExecutor pool thread. | ||
| Future<ContainerCommandResponseProto> future = getChunkExecutor( | ||
| requestProto.getWriteChunk()).submit(() -> { | ||
| try { | ||
| try { | ||
| checkContainerHealthy(write.getBlockID().getContainerID(), true); | ||
| } catch (StorageContainerException e) { | ||
| return ContainerUtils.logAndReturnError(LOG, e, requestProto); | ||
| ContainerCommandResponseProto result = ContainerUtils.logAndReturnError(LOG, e, requestProto); | ||
| handleCommandResult(requestProto, entryIndex, startTime, result, write, raftFuture); | ||
| return result; | ||
| } | ||
| metrics.recordWriteStateMachineQueueingLatencyNs( | ||
| Time.monotonicNowNanos() - startTime); | ||
| return dispatchCommand(requestProto, context); | ||
| ContainerCommandResponseProto result = dispatchCommand(requestProto, context); | ||
| handleCommandResult(requestProto, entryIndex, startTime, result, write, raftFuture); | ||
| return result; | ||
| } catch (Exception e) { | ||
| LOG.error("{}: writeChunk writeStateMachineData failed: blockId" + | ||
| "{} logIndex {} chunkName {}", getGroupId(), write.getBlockID(), | ||
| "{} logIndex {} chunkName {}", getGroupId(), write.getBlockID(), | ||
|
||
| entryIndex, write.getChunkData().getChunkName(), e); | ||
| metrics.incNumWriteDataFails(); | ||
| // write chunks go in parallel. It's possible that one write chunk | ||
|
|
@@ -588,55 +633,91 @@ private CompletableFuture<Message> writeStateMachineData( | |
| stateMachineHealthy.set(false); | ||
| raftFuture.completeExceptionally(e); | ||
| throw e; | ||
| } finally { | ||
| // Remove the future once it finishes execution from the | ||
| writeChunkFutureMap.remove(entryIndex); | ||
| } | ||
| }, getChunkExecutor(requestProto.getWriteChunk())); | ||
| }); | ||
|
|
||
| writeChunkFutureMap.put(entryIndex, writeChunkFuture); | ||
| writeChunkFutureMap.put(entryIndex, new WriteFutures(future, raftFuture, startTime)); | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug("{}: writeChunk writeStateMachineData : blockId" + | ||
| "{} logIndex {} chunkName {}", getGroupId(), write.getBlockID(), | ||
| entryIndex, write.getChunkData().getChunkName()); | ||
| } | ||
| // Remove the future once it finishes execution from the | ||
| // writeChunkFutureMap. | ||
| writeChunkFuture.thenApply(r -> { | ||
| if (r.getResult() != ContainerProtos.Result.SUCCESS | ||
| && r.getResult() != ContainerProtos.Result.CONTAINER_NOT_OPEN | ||
| && r.getResult() != ContainerProtos.Result.CLOSED_CONTAINER_IO | ||
| // After concurrent flushes are allowed on the same key, chunk file inconsistencies can happen and | ||
| // that should not crash the pipeline. | ||
| && r.getResult() != ContainerProtos.Result.CHUNK_FILE_INCONSISTENCY) { | ||
| StorageContainerException sce = | ||
| new StorageContainerException(r.getMessage(), r.getResult()); | ||
| LOG.error(getGroupId() + ": writeChunk writeStateMachineData failed: blockId" + | ||
| return raftFuture; | ||
| } | ||
|
|
||
| private void handleCommandResult(ContainerCommandRequestProto requestProto, long entryIndex, long startTime, | ||
| ContainerCommandResponseProto r, WriteChunkRequestProto write, | ||
| CompletableFuture<Message> raftFuture) { | ||
| if (r.getResult() != ContainerProtos.Result.SUCCESS | ||
| && r.getResult() != ContainerProtos.Result.CONTAINER_NOT_OPEN | ||
| && r.getResult() != ContainerProtos.Result.CLOSED_CONTAINER_IO | ||
| // After concurrent flushes are allowed on the same key, chunk file inconsistencies can happen and | ||
| // that should not crash the pipeline. | ||
| && r.getResult() != ContainerProtos.Result.CHUNK_FILE_INCONSISTENCY) { | ||
| StorageContainerException sce = | ||
| new StorageContainerException(r.getMessage(), r.getResult()); | ||
| LOG.error(getGroupId() + ": writeChunk writeStateMachineData failed: blockId" + | ||
| write.getBlockID() + " logIndex " + entryIndex + " chunkName " + | ||
| write.getChunkData().getChunkName() + " Error message: " + | ||
| r.getMessage() + " Container Result: " + r.getResult()); | ||
| metrics.incNumWriteDataFails(); | ||
| // If the write fails currently we mark the stateMachine as unhealthy. | ||
| // This leads to pipeline close. Any change in that behavior requires | ||
| // handling the entry for the write chunk in cache. | ||
| stateMachineHealthy.set(false); | ||
| unhealthyContainers.add(write.getBlockID().getContainerID()); | ||
| raftFuture.completeExceptionally(sce); | ||
| } else { | ||
| metrics.incNumBytesWrittenCount( | ||
| requestProto.getWriteChunk().getChunkData().getLen()); | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug(getGroupId() + | ||
| ": writeChunk writeStateMachineData completed: blockId" + | ||
| write.getBlockID() + " logIndex " + entryIndex + " chunkName " + | ||
| write.getChunkData().getChunkName() + " Error message: " + | ||
| r.getMessage() + " Container Result: " + r.getResult()); | ||
| metrics.incNumWriteDataFails(); | ||
| // If the write fails currently we mark the stateMachine as unhealthy. | ||
| // This leads to pipeline close. Any change in that behavior requires | ||
| // handling the entry for the write chunk in cache. | ||
| stateMachineHealthy.set(false); | ||
| unhealthyContainers.add(write.getBlockID().getContainerID()); | ||
| raftFuture.completeExceptionally(sce); | ||
| } else { | ||
| metrics.incNumBytesWrittenCount( | ||
| requestProto.getWriteChunk().getChunkData().getLen()); | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug(getGroupId() + | ||
| ": writeChunk writeStateMachineData completed: blockId" + | ||
| write.getBlockID() + " logIndex " + entryIndex + " chunkName " + | ||
| write.getChunkData().getChunkName()); | ||
| } | ||
| raftFuture.complete(r::toByteString); | ||
| metrics.recordWriteStateMachineCompletionNs( | ||
| Time.monotonicNowNanos() - startTime); | ||
| write.getChunkData().getChunkName()); | ||
| } | ||
| raftFuture.complete(r::toByteString); | ||
| metrics.recordWriteStateMachineCompletionNs( | ||
| Time.monotonicNowNanos() - startTime); | ||
| } | ||
| } | ||
|
|
||
| writeChunkFutureMap.remove(entryIndex); | ||
| return r; | ||
| }); | ||
| return raftFuture; | ||
| private void validateLongRunningWrite() throws StorageContainerException { | ||
| // get min valid write chunk operation's future context | ||
| Map.Entry<Long, WriteFutures> writeFutureContextEntry = null; | ||
| for (boolean found = false; !found;) { | ||
| writeFutureContextEntry = writeChunkFutureMap.firstEntry(); | ||
| if (null == writeFutureContextEntry) { | ||
| return; | ||
| } | ||
| if (writeFutureContextEntry.getValue().getWriteChunkFuture().isDone()) { | ||
| // there is a possibility that writeChunkFutureMap may have dangling entry, as remove is done before add future | ||
| writeChunkFutureMap.remove(writeFutureContextEntry.getKey()); | ||
| } else { | ||
| found = true; | ||
| } | ||
| } | ||
| if (null == writeFutureContextEntry) { | ||
| return; | ||
| } | ||
|
||
| // validate for timeout in milli second | ||
| long waitTime = Time.monotonicNowNanos() - writeFutureContextEntry.getValue().getStartTime(); | ||
| if (waitTime > writeChunkWaitMaxNs) { | ||
| LOG.error("Write chunk has taken {}ns crossing threshold {}ns for index {} groupId {}", waitTime, | ||
| writeChunkWaitMaxNs, writeFutureContextEntry.getKey(), getGroupId()); | ||
| stateMachineHealthy.set(false); | ||
| writeChunkFutureMap.forEach((key, value) -> { | ||
| LOG.error("Cancelling write chunk due to timeout {}ns crossing {}ns for index {}, groupId {}", waitTime, | ||
|
||
| writeChunkWaitMaxNs, key, getGroupId()); | ||
| value.getWriteChunkFuture().cancel(true); | ||
| }); | ||
| throw new StorageContainerException("Write chunk has taken " + waitTime + "ns crossing threshold " | ||
| + writeChunkWaitMaxNs + "ns for index " + writeFutureContextEntry.getKey() + " groupId " + getGroupId(), | ||
| ContainerProtos.Result.CONTAINER_INTERNAL_ERROR); | ||
| } | ||
| } | ||
|
|
||
| private StateMachine.DataChannel getStreamDataChannel( | ||
|
|
@@ -819,9 +900,13 @@ private ByteString readStateMachineData( | |
| */ | ||
| @Override | ||
| public CompletableFuture<Void> flush(long index) { | ||
| return CompletableFuture.allOf( | ||
| writeChunkFutureMap.entrySet().stream().filter(x -> x.getKey() <= index) | ||
| .map(Map.Entry::getValue).toArray(CompletableFuture[]::new)); | ||
| final SortedMap<Long, WriteFutures> head = writeChunkFutureMap.headMap(index + 1); | ||
|
||
| if (head.isEmpty()) { | ||
| return CompletableFuture.completedFuture(null); | ||
| } | ||
| return CompletableFuture.allOf(head.values().stream() | ||
| .map(WriteFutures::getRaftFuture) | ||
| .toArray(CompletableFuture[]::new)); | ||
| } | ||
|
|
||
| /** | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
TYPO:
1000_1000_1000Lshould be1000_000_000LThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks, updated