Skip to content

Commit 05d71ad

Browse files
KAFKA-19476: Concurrent execution fixes for lock timeout and lso movement (#20286)
The PR fixes following: 1. In case share partition arrive at a state which should be treated as final state of that batch/offset (example - LSO movement which causes offset/batch to be ARCHIVED permanently), the result of pending write state RPCs for that offset/batch override the ARCHIVED state. Hence track such updates and apply when transition is completed. 2. If an acquisition lock timeout occurs while an offset/batch is undergoing transition followed by write state RPC failure, then respective batch/offset can land in a scenario where the offset stays in ACQUIRED state with no acquisition lock timeout task. 3. If a timer task is cancelled, but due to concurrent execution of timer task and acknowledgement, there can be a scenario when timer task has processed post cancellation. Hence it can mark the offset/batch re-avaialble despite already acknowledged. Reviewers: Andrew Schofield <[email protected]>, Abhinav Dixit <[email protected]>
1 parent b909544 commit 05d71ad

File tree

6 files changed

+425
-42
lines changed

6 files changed

+425
-42
lines changed

core/src/main/java/kafka/server/share/SharePartition.java

Lines changed: 46 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -241,6 +241,12 @@ enum SharePartitionState {
241241
*/
242242
private final AcquisitionLockTimeoutHandler timeoutHandler;
243243

244+
/**
245+
* The replica manager is used to check to see if any delayed share fetch request can be completed because of data
246+
* availability due to acquisition lock timeout.
247+
*/
248+
private final ReplicaManager replicaManager;
249+
244250
/**
245251
* The share partition start offset specifies the partition start offset from which the records
246252
* are cached in the cachedState of the sharePartition.
@@ -295,12 +301,6 @@ enum SharePartitionState {
295301
*/
296302
private long fetchLockIdleDurationMs;
297303

298-
/**
299-
* The replica manager is used to check to see if any delayed share fetch request can be completed because of data
300-
* availability due to acquisition lock timeout.
301-
*/
302-
private final ReplicaManager replicaManager;
303-
304304
SharePartition(
305305
String groupId,
306306
TopicIdPartition topicIdPartition,
@@ -1245,10 +1245,7 @@ private boolean archivePerOffsetBatchRecords(InFlightBatch inFlightBatch,
12451245
continue;
12461246
}
12471247

1248-
offsetState.getValue().archive(EMPTY_MEMBER_ID);
1249-
if (initialState == RecordState.ACQUIRED) {
1250-
offsetState.getValue().cancelAndClearAcquisitionLockTimeoutTask();
1251-
}
1248+
offsetState.getValue().archive();
12521249
isAnyOffsetArchived = true;
12531250
}
12541251
return isAnyOffsetArchived;
@@ -1263,10 +1260,7 @@ private boolean archiveCompleteBatch(InFlightBatch inFlightBatch, RecordState in
12631260
log.trace("Archiving complete batch: {} for the share partition: {}-{}", inFlightBatch, groupId, topicIdPartition);
12641261
if (inFlightBatch.batchState() == initialState) {
12651262
// Change the state of complete batch since the same state exists for the entire inFlight batch.
1266-
inFlightBatch.archiveBatch(EMPTY_MEMBER_ID);
1267-
if (initialState == RecordState.ACQUIRED) {
1268-
inFlightBatch.cancelAndClearAcquisitionLockTimeoutTask();
1269-
}
1263+
inFlightBatch.archiveBatch();
12701264
return true;
12711265
}
12721266
} finally {
@@ -1799,6 +1793,12 @@ private Optional<Throwable> acknowledgeBatchRecords(
17991793
if (throwable.isPresent()) {
18001794
return throwable;
18011795
}
1796+
1797+
if (inFlightBatch.batchHasOngoingStateTransition()) {
1798+
log.debug("The batch has on-going transition, batch: {} for the share "
1799+
+ "partition: {}-{}", inFlightBatch, groupId, topicIdPartition);
1800+
return Optional.of(new InvalidRecordStateException("The record state is invalid. The acknowledgement of delivery could not be completed."));
1801+
}
18021802
}
18031803

18041804
// Determine if the in-flight batch is a full match from the request batch.
@@ -1899,7 +1899,15 @@ private Optional<Throwable> acknowledgePerOffsetBatchRecords(
18991899
+ " partition: {}-{}", offsetState.getKey(), inFlightBatch, groupId,
19001900
topicIdPartition);
19011901
return Optional.of(new InvalidRecordStateException(
1902-
"The batch cannot be acknowledged. The offset is not acquired."));
1902+
"The offset cannot be acknowledged. The offset is not acquired."));
1903+
}
1904+
1905+
if (offsetState.getValue().hasOngoingStateTransition()) {
1906+
log.debug("The offset has on-going transition, offset: {} batch: {} for the share"
1907+
+ " partition: {}-{}", offsetState.getKey(), inFlightBatch, groupId,
1908+
topicIdPartition);
1909+
return Optional.of(new InvalidRecordStateException(
1910+
"The record state is invalid. The acknowledgement of delivery could not be completed."));
19031911
}
19041912

19051913
// Check if member id is the owner of the offset.
@@ -2044,7 +2052,12 @@ void rollbackOrProcessStateUpdates(
20442052
// Log in DEBUG to avoid flooding of logs for a faulty client.
20452053
log.debug("Request failed for updating state, rollback any changed state"
20462054
+ " for the share partition: {}-{}", groupId, topicIdPartition);
2047-
updatedStates.forEach(state -> state.completeStateTransition(false));
2055+
updatedStates.forEach(state -> {
2056+
state.completeStateTransition(false);
2057+
if (state.state() == RecordState.AVAILABLE) {
2058+
updateFindNextFetchOffset(true);
2059+
}
2060+
});
20482061
future.completeExceptionally(throwable);
20492062
return;
20502063
}
@@ -2067,7 +2080,14 @@ void rollbackOrProcessStateUpdates(
20672080
if (exception != null) {
20682081
log.debug("Failed to write state to persister for the share partition: {}-{}",
20692082
groupId, topicIdPartition, exception);
2070-
updatedStates.forEach(state -> state.completeStateTransition(false));
2083+
// In case of failure when transition state is rolled back then it should be rolled
2084+
// back to ACQUIRED state, unless acquisition lock for the state has expired.
2085+
updatedStates.forEach(state -> {
2086+
state.completeStateTransition(false);
2087+
if (state.state() == RecordState.AVAILABLE) {
2088+
updateFindNextFetchOffset(true);
2089+
}
2090+
});
20712091
future.completeExceptionally(exception);
20722092
return;
20732093
}
@@ -2076,8 +2096,6 @@ void rollbackOrProcessStateUpdates(
20762096
groupId, topicIdPartition);
20772097
updatedStates.forEach(state -> {
20782098
state.completeStateTransition(true);
2079-
// Cancel the acquisition lock timeout task for the state since it is acknowledged/released successfully.
2080-
state.cancelAndClearAcquisitionLockTimeoutTask();
20812099
if (state.state() == RecordState.AVAILABLE) {
20822100
updateFindNextFetchOffset(true);
20832101
}
@@ -2389,10 +2407,18 @@ private AcquisitionLockTimerTask acquisitionLockTimerTask(
23892407
}
23902408

23912409
private AcquisitionLockTimeoutHandler releaseAcquisitionLockOnTimeout() {
2392-
return (memberId, firstOffset, lastOffset) -> {
2410+
return (memberId, firstOffset, lastOffset, timerTask) -> {
23932411
List<PersisterStateBatch> stateBatches;
23942412
lock.writeLock().lock();
23952413
try {
2414+
// Check if timer task is already cancelled. This can happen when concurrent requests
2415+
// happen to acknowledge in-flight state and timeout handler is waiting for the lock
2416+
// but already cancelled.
2417+
if (timerTask.isCancelled()) {
2418+
log.debug("Timer task is already cancelled, not executing further.");
2419+
return;
2420+
}
2421+
23962422
Map.Entry<Long, InFlightBatch> floorOffset = cachedState.floorEntry(firstOffset);
23972423
if (floorOffset == null) {
23982424
log.error("Base offset {} not found for share partition: {}-{}", firstOffset, groupId, topicIdPartition);

0 commit comments

Comments
 (0)