-
Notifications
You must be signed in to change notification settings - Fork 14.1k
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
KAFKA-18058: Share group state record pruning impl. #18014
Changes from all commits
546eda7
dd575d4
3dc1798
9d19c0b
87f5a1d
d566e22
6ae8337
c49b7db
f616f85
bea9566
b0d4cca
9a342c2
e77c16d
6d889bc
b30df09
f56477b
055ef21
7e62122
18a44e1
74dcd45
f254399
a9a986e
b68b014
1cc890a
e1a38ab
ae6333a
8244bdc
c798254
75eb4d2
be0310f
a5aa93f
78d8c49
9714ce6
7e420cb
e3db1dc
1d60249
502b5e7
d55abdd
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 |
---|---|---|
|
@@ -20,13 +20,14 @@ import kafka.server.ReplicaManager | |
import org.apache.kafka.common.TopicPartition | ||
import org.apache.kafka.common.compress.Compression | ||
import org.apache.kafka.common.errors.NotLeaderOrFollowerException | ||
import org.apache.kafka.common.message.DeleteRecordsResponseData.DeleteRecordsPartitionResult | ||
import org.apache.kafka.common.protocol.{ApiKeys, Errors} | ||
import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, SimpleRecord} | ||
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse | ||
import org.apache.kafka.coordinator.common.runtime.PartitionWriter | ||
import org.apache.kafka.storage.internals.log.{AppendOrigin, LogConfig, VerificationGuard} | ||
import org.apache.kafka.test.TestUtils.assertFutureThrows | ||
import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} | ||
import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertThrows, assertTrue} | ||
import org.junit.jupiter.api.Test | ||
import org.junit.jupiter.params.ParameterizedTest | ||
import org.junit.jupiter.params.provider.EnumSource | ||
|
@@ -238,4 +239,83 @@ class CoordinatorPartitionWriterTest { | |
batch | ||
)) | ||
} | ||
|
||
@Test | ||
def testDeleteRecordsResponseContainsError(): Unit = { | ||
val replicaManager = mock(classOf[ReplicaManager]) | ||
val partitionRecordWriter = new CoordinatorPartitionWriter( | ||
replicaManager | ||
) | ||
|
||
val callbackCapture: ArgumentCaptor[Map[TopicPartition, DeleteRecordsPartitionResult] => Unit] = | ||
ArgumentCaptor.forClass(classOf[Map[TopicPartition, DeleteRecordsPartitionResult] => Unit]) | ||
|
||
// Response contains error. | ||
when(replicaManager.deleteRecords( | ||
ArgumentMatchers.anyLong(), | ||
ArgumentMatchers.any(), | ||
callbackCapture.capture(), | ||
ArgumentMatchers.eq(true) | ||
)).thenAnswer { _ => | ||
callbackCapture.getValue.apply(Map( | ||
new TopicPartition("random-topic", 0) -> new DeleteRecordsPartitionResult() | ||
.setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code | ||
))) | ||
} | ||
|
||
partitionRecordWriter.deleteRecords( | ||
new TopicPartition("random-topic", 0), | ||
10L | ||
).whenComplete { (_, exp) => | ||
assertEquals(Errors.NOT_LEADER_OR_FOLLOWER.exception, exp) | ||
} | ||
|
||
// Empty response | ||
when(replicaManager.deleteRecords( | ||
ArgumentMatchers.anyLong(), | ||
ArgumentMatchers.any(), | ||
callbackCapture.capture(), | ||
ArgumentMatchers.eq(true) | ||
)).thenAnswer { _ => | ||
callbackCapture.getValue.apply(Map[TopicPartition, DeleteRecordsPartitionResult]()) | ||
} | ||
|
||
partitionRecordWriter.deleteRecords( | ||
new TopicPartition("random-topic", 0), | ||
10L | ||
).whenComplete { (_, exp) => | ||
assertTrue(exp.isInstanceOf[IllegalStateException]) | ||
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. If a topic doesn't exist, we should get an unknown topic exception. |
||
} | ||
} | ||
|
||
@Test | ||
def testDeleteRecordsSuccess(): Unit = { | ||
val replicaManager = mock(classOf[ReplicaManager]) | ||
val partitionRecordWriter = new CoordinatorPartitionWriter( | ||
replicaManager | ||
) | ||
|
||
val callbackCapture: ArgumentCaptor[Map[TopicPartition, DeleteRecordsPartitionResult] => Unit] = | ||
ArgumentCaptor.forClass(classOf[Map[TopicPartition, DeleteRecordsPartitionResult] => Unit]) | ||
|
||
// response contains error | ||
when(replicaManager.deleteRecords( | ||
ArgumentMatchers.anyLong(), | ||
ArgumentMatchers.any(), | ||
callbackCapture.capture(), | ||
ArgumentMatchers.eq(true) | ||
)).thenAnswer { _ => | ||
callbackCapture.getValue.apply(Map( | ||
new TopicPartition("random-topic", 0) -> new DeleteRecordsPartitionResult() | ||
.setErrorCode(Errors.NONE.code) | ||
)) | ||
} | ||
|
||
partitionRecordWriter.deleteRecords( | ||
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. ditto. |
||
new TopicPartition("random-topic", 0), | ||
10L | ||
).whenComplete { (_, exp) => | ||
assertNull(exp) | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -34,7 +34,8 @@ import org.apache.kafka.common.{DirectoryId, IsolationLevel, Node, TopicIdPartit | |
import org.apache.kafka.common.compress.Compression | ||
import org.apache.kafka.common.config.TopicConfig | ||
import org.apache.kafka.common.errors.{InvalidPidMappingException, KafkaStorageException} | ||
import org.apache.kafka.common.message.LeaderAndIsrRequestData | ||
import org.apache.kafka.common.internals.Topic | ||
import org.apache.kafka.common.message.{DeleteRecordsResponseData, LeaderAndIsrRequestData} | ||
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState | ||
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset | ||
import org.apache.kafka.common.message.StopReplicaRequestData.StopReplicaPartitionState | ||
|
@@ -6660,6 +6661,61 @@ class ReplicaManagerTest { | |
} | ||
} | ||
|
||
@Test | ||
def testDeleteRecordsInternalTopicDeleteDisallowed(): Unit = { | ||
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. There are lots of existing usage of 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. perhaps |
||
val localId = 1 | ||
val topicPartition0 = new TopicIdPartition(FOO_UUID, 0, Topic.GROUP_METADATA_TOPIC_NAME) | ||
val directoryEventHandler = mock(classOf[DirectoryEventHandler]) | ||
|
||
val rm = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), localId, setupLogDirMetaProperties = true, directoryEventHandler = directoryEventHandler) | ||
val directoryIds = rm.logManager.directoryIdsSet.toList | ||
assertEquals(directoryIds.size, 2) | ||
val leaderTopicsDelta: TopicsDelta = topicsCreateDelta(localId, isStartIdLeader = true, directoryIds = directoryIds) | ||
val (partition: Partition, _) = rm.getOrCreatePartition(topicPartition0.topicPartition(), leaderTopicsDelta, FOO_UUID).get | ||
partition.makeLeader(leaderAndIsrPartitionState(topicPartition0.topicPartition(), 1, localId, Seq(1, 2)), | ||
new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints.asJava), | ||
None) | ||
|
||
def callback(responseStatus: Map[TopicPartition, DeleteRecordsResponseData.DeleteRecordsPartitionResult]): Unit = { | ||
assert(responseStatus.values.head.errorCode == Errors.INVALID_TOPIC_EXCEPTION.code) | ||
} | ||
|
||
// default internal topics delete disabled | ||
rm.deleteRecords( | ||
timeout = 0L, | ||
Map[TopicPartition, Long](topicPartition0.topicPartition() -> 10L), | ||
responseCallback = callback | ||
) | ||
} | ||
|
||
@Test | ||
def testDeleteRecordsInternalTopicDeleteAllowed(): Unit = { | ||
val localId = 1 | ||
val topicPartition0 = new TopicIdPartition(FOO_UUID, 0, Topic.GROUP_METADATA_TOPIC_NAME) | ||
val directoryEventHandler = mock(classOf[DirectoryEventHandler]) | ||
|
||
val rm = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), localId, setupLogDirMetaProperties = true, directoryEventHandler = directoryEventHandler) | ||
val directoryIds = rm.logManager.directoryIdsSet.toList | ||
assertEquals(directoryIds.size, 2) | ||
val leaderTopicsDelta: TopicsDelta = topicsCreateDelta(localId, isStartIdLeader = true, directoryIds = directoryIds) | ||
val (partition: Partition, _) = rm.getOrCreatePartition(topicPartition0.topicPartition(), leaderTopicsDelta, FOO_UUID).get | ||
partition.makeLeader(leaderAndIsrPartitionState(topicPartition0.topicPartition(), 1, localId, Seq(1, 2)), | ||
new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints.asJava), | ||
None) | ||
|
||
def callback(responseStatus: Map[TopicPartition, DeleteRecordsResponseData.DeleteRecordsPartitionResult]): Unit = { | ||
assert(responseStatus.values.head.errorCode == Errors.NONE.code) | ||
} | ||
|
||
// internal topics delete allowed | ||
rm.deleteRecords( | ||
timeout = 0L, | ||
Map[TopicPartition, Long](topicPartition0.topicPartition() -> 0L), | ||
responseCallback = callback, | ||
allowInternalTopicDeletion = true | ||
) | ||
} | ||
|
||
private def readFromLogWithOffsetOutOfRange(tp: TopicPartition): Seq[(TopicIdPartition, LogReadResult)] = { | ||
val replicaManager = setupReplicaManagerWithMockedPurgatories(new MockTimer(time), aliveBrokerIds = Seq(0, 1, 2), enableRemoteStorage = true, shouldMockLog = true) | ||
try { | ||
|
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.
Let's add a unit test for this new flag in ReplicaManagerTest.