-
Notifications
You must be signed in to change notification settings - Fork 8.9k
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
HDDS-1610. applyTransaction failure should not be lost on restart. #1226
Closed
Closed
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
a6f60eb
HDDS-1610. applyTransaction failure should not be lost on restart.
bshashikant 529a05b
Addressed review comments.
bshashikant 412a7ff
Addressed review comments and checkstyle issues.
bshashikant 0d301a0
Addressed related test failures
bshashikant d559ba0
Fix the test failure related to CSMMetrics.
bshashikant File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -34,6 +34,7 @@ | |
import org.apache.hadoop.util.Time; | ||
import org.apache.ratis.proto.RaftProtos.RaftPeerRole; | ||
import org.apache.ratis.protocol.RaftGroupId; | ||
import org.apache.ratis.protocol.StateMachineException; | ||
import org.apache.ratis.server.RaftServer; | ||
import org.apache.ratis.server.impl.RaftServerProxy; | ||
import org.apache.ratis.server.protocol.TermIndex; | ||
|
@@ -83,6 +84,7 @@ | |
import java.util.concurrent.Semaphore; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.concurrent.ExecutionException; | ||
import java.util.concurrent.atomic.AtomicBoolean; | ||
import java.util.stream.Collectors; | ||
import java.util.Set; | ||
import java.util.concurrent.ConcurrentSkipListSet; | ||
|
@@ -147,6 +149,7 @@ public class ContainerStateMachine extends BaseStateMachine { | |
private final Cache<Long, ByteString> stateMachineDataCache; | ||
private final boolean isBlockTokenEnabled; | ||
private final TokenVerifier tokenVerifier; | ||
private final AtomicBoolean isStateMachineHealthy; | ||
|
||
private final Semaphore applyTransactionSemaphore; | ||
/** | ||
|
@@ -184,6 +187,7 @@ public ContainerStateMachine(RaftGroupId gid, ContainerDispatcher dispatcher, | |
ScmConfigKeys. | ||
DFS_CONTAINER_RATIS_STATEMACHINE_MAX_PENDING_APPLY_TXNS_DEFAULT); | ||
applyTransactionSemaphore = new Semaphore(maxPendingApplyTransactions); | ||
isStateMachineHealthy = new AtomicBoolean(true); | ||
this.executors = new ExecutorService[numContainerOpExecutors]; | ||
for (int i = 0; i < numContainerOpExecutors; i++) { | ||
final int index = i; | ||
|
@@ -265,6 +269,14 @@ public void persistContainerSet(OutputStream out) throws IOException { | |
public long takeSnapshot() throws IOException { | ||
TermIndex ti = getLastAppliedTermIndex(); | ||
long startTime = Time.monotonicNow(); | ||
if (!isStateMachineHealthy.get()) { | ||
String msg = | ||
"Failed to take snapshot " + " for " + gid + " as the stateMachine" | ||
+ " is unhealthy. The last applied index is at " + ti; | ||
StateMachineException sme = new StateMachineException(msg); | ||
LOG.error(msg); | ||
throw sme; | ||
} | ||
if (ti != null && ti.getIndex() != RaftLog.INVALID_LOG_INDEX) { | ||
final File snapshotFile = | ||
storage.getSnapshotFile(ti.getTerm(), ti.getIndex()); | ||
|
@@ -275,12 +287,12 @@ public long takeSnapshot() throws IOException { | |
// make sure the snapshot file is synced | ||
fos.getFD().sync(); | ||
} catch (IOException ioe) { | ||
LOG.info("{}: Failed to write snapshot at:{} file {}", gid, ti, | ||
LOG.error("{}: Failed to write snapshot at:{} file {}", gid, ti, | ||
snapshotFile); | ||
throw ioe; | ||
} | ||
LOG.info("{}: Finished taking a snapshot at:{} file:{} time:{}", | ||
gid, ti, snapshotFile, (Time.monotonicNow() - startTime)); | ||
LOG.info("{}: Finished taking a snapshot at:{} file:{} time:{}", gid, ti, | ||
snapshotFile, (Time.monotonicNow() - startTime)); | ||
return ti.getIndex(); | ||
} | ||
return -1; | ||
|
@@ -385,17 +397,12 @@ private ContainerCommandResponseProto dispatchCommand( | |
return response; | ||
} | ||
|
||
private ContainerCommandResponseProto runCommandGetResponse( | ||
private ContainerCommandResponseProto runCommand( | ||
ContainerCommandRequestProto requestProto, | ||
DispatcherContext context) { | ||
return dispatchCommand(requestProto, context); | ||
} | ||
|
||
private Message runCommand(ContainerCommandRequestProto requestProto, | ||
DispatcherContext context) { | ||
return runCommandGetResponse(requestProto, context)::toByteString; | ||
} | ||
|
||
private ExecutorService getCommandExecutor( | ||
ContainerCommandRequestProto requestProto) { | ||
int executorId = (int)(requestProto.getContainerID() % executors.length); | ||
|
@@ -425,7 +432,7 @@ private CompletableFuture<Message> handleWriteChunk( | |
// thread. | ||
CompletableFuture<ContainerCommandResponseProto> writeChunkFuture = | ||
CompletableFuture.supplyAsync(() -> | ||
runCommandGetResponse(requestProto, context), chunkExecutor); | ||
runCommand(requestProto, context), chunkExecutor); | ||
|
||
CompletableFuture<Message> raftFuture = new CompletableFuture<>(); | ||
|
||
|
@@ -502,7 +509,8 @@ public CompletableFuture<Message> query(Message request) { | |
metrics.incNumQueryStateMachineOps(); | ||
final ContainerCommandRequestProto requestProto = | ||
getContainerCommandRequestProto(request.getContent()); | ||
return CompletableFuture.completedFuture(runCommand(requestProto, null)); | ||
return CompletableFuture | ||
.completedFuture(runCommand(requestProto, null)::toByteString); | ||
} catch (IOException e) { | ||
metrics.incNumQueryStateMachineFails(); | ||
return completeExceptionally(e); | ||
|
@@ -674,30 +682,58 @@ public CompletableFuture<Message> applyTransaction(TransactionContext trx) { | |
if (cmdType == Type.WriteChunk || cmdType ==Type.PutSmallFile) { | ||
builder.setCreateContainerSet(createContainerSet); | ||
} | ||
CompletableFuture<Message> applyTransactionFuture = | ||
new CompletableFuture<>(); | ||
// Ensure the command gets executed in a separate thread than | ||
// stateMachineUpdater thread which is calling applyTransaction here. | ||
CompletableFuture<Message> future = CompletableFuture | ||
.supplyAsync(() -> runCommand(requestProto, builder.build()), | ||
CompletableFuture<ContainerCommandResponseProto> future = | ||
CompletableFuture.supplyAsync( | ||
() -> runCommand(requestProto, builder.build()), | ||
getCommandExecutor(requestProto)); | ||
|
||
future.thenAccept(m -> { | ||
future.thenApply(r -> { | ||
if (trx.getServerRole() == RaftPeerRole.LEADER) { | ||
long startTime = (long) trx.getStateMachineContext(); | ||
metrics.incPipelineLatency(cmdType, | ||
Time.monotonicNowNanos() - startTime); | ||
} | ||
|
||
final Long previous = | ||
applyTransactionCompletionMap | ||
if (r.getResult() != ContainerProtos.Result.SUCCESS) { | ||
StorageContainerException sce = | ||
new StorageContainerException(r.getMessage(), r.getResult()); | ||
LOG.error( | ||
"gid {} : ApplyTransaction failed. cmd {} logIndex {} msg : " | ||
+ "{} Container Result: {}", gid, r.getCmdType(), index, | ||
r.getMessage(), r.getResult()); | ||
metrics.incNumApplyTransactionsFails(); | ||
// Since the applyTransaction now is completed exceptionally, | ||
// before any further snapshot is taken , the exception will be | ||
// caught in stateMachineUpdater in Ratis and ratis server will | ||
// shutdown. | ||
applyTransactionFuture.completeExceptionally(sce); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. lets move the ratisServer.handleApplyTransactionFailure(gid, trx.getServerRole()); as the last line in the if block. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Addressed in the latest patch. |
||
isStateMachineHealthy.compareAndSet(true, false); | ||
ratisServer.handleApplyTransactionFailure(gid, trx.getServerRole()); | ||
} else { | ||
LOG.debug( | ||
"gid {} : ApplyTransaction completed. cmd {} logIndex {} msg : " | ||
+ "{} Container Result: {}", gid, r.getCmdType(), index, | ||
r.getMessage(), r.getResult()); | ||
applyTransactionFuture.complete(r::toByteString); | ||
if (cmdType == Type.WriteChunk || cmdType == Type.PutSmallFile) { | ||
metrics.incNumBytesCommittedCount( | ||
requestProto.getWriteChunk().getChunkData().getLen()); | ||
} | ||
// add the entry to the applyTransactionCompletionMap only if the | ||
// stateMachine is healthy i.e, there has been no applyTransaction | ||
// failures before. | ||
if (isStateMachineHealthy.get()) { | ||
final Long previous = applyTransactionCompletionMap | ||
.put(index, trx.getLogEntry().getTerm()); | ||
Preconditions.checkState(previous == null); | ||
if (cmdType == Type.WriteChunk || cmdType == Type.PutSmallFile) { | ||
metrics.incNumBytesCommittedCount( | ||
requestProto.getWriteChunk().getChunkData().getLen()); | ||
Preconditions.checkState(previous == null); | ||
updateLastApplied(); | ||
} | ||
} | ||
updateLastApplied(); | ||
return applyTransactionFuture; | ||
}).whenComplete((r, t) -> applyTransactionSemaphore.release()); | ||
return future; | ||
return applyTransactionFuture; | ||
} catch (IOException | InterruptedException e) { | ||
metrics.incNumApplyTransactionsFails(); | ||
return completeExceptionally(e); | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
lets log this as well.
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.
Addressed in the latest patch.