-
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 9 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 |
---|---|---|
|
@@ -19,6 +19,7 @@ package kafka.coordinator.group | |
import kafka.cluster.PartitionListener | ||
import kafka.server.{ReplicaManager, defaultError, genericError} | ||
import org.apache.kafka.common.TopicPartition | ||
import org.apache.kafka.common.message.DeleteRecordsResponseData.DeleteRecordsPartitionResult | ||
import org.apache.kafka.common.protocol.Errors | ||
import org.apache.kafka.common.record.{MemoryRecords, RecordBatch} | ||
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse | ||
|
@@ -165,4 +166,21 @@ class CoordinatorPartitionWriter( | |
// Required offset. | ||
partitionResult.lastOffset + 1 | ||
} | ||
|
||
override def deleteRecords(tp: TopicPartition, deleteBeforeOffset: Long, allowInternalTopicDeletion: Boolean): Unit = { | ||
var deleteResults: Map[TopicPartition, DeleteRecordsPartitionResult] = Map.empty | ||
replicaManager.deleteRecords( | ||
timeout = 0L, | ||
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. I have doubts about using timeout=0 here. My understanding is that the it will delete records from the local logs but it wont wait on the replication to acknowledges the deletion from the replicas. Hence, I suppose that it will always throw an error. I suppose that we don't really care about the replication in this case but it may spam the logs with unnecessary errors. 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. This is the same as append records https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/coordinator/group/CoordinatorPartitionWriter.scala#L144 will add 30 second timeout nevertheless 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. You cannot compare with append records in this case because they don't work the sam way. In append records, the key is that we use With the timeout, I believe that the following code will actually throw a
Could you please double check? One option may be to make delete records return a future and complete it when the callback returns. This would also allow you to not trigger the next purge while the current one is not completed. |
||
offsetPerPartition = Map(tp -> deleteBeforeOffset), | ||
responseCallback = results => deleteResults = results, | ||
allowInternalTopicDeletion | ||
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. I wonder if we could just set it to |
||
) | ||
|
||
val partitionResult = deleteResults.getOrElse(tp, | ||
throw new IllegalStateException(s"Delete status $deleteResults should have partition $tp.")) | ||
|
||
if (partitionResult.errorCode() != Errors.NONE.code()) { | ||
throw Errors.forCode(partitionResult.errorCode()).exception() | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1172,11 +1172,11 @@ class ReplicaManager(val config: KafkaConfig, | |
* Delete records on leader replicas of the partition, and wait for delete records operation be propagated to other replicas; | ||
* the callback function will be triggered either when timeout or logStartOffset of all live replicas have reached the specified offset | ||
*/ | ||
private def deleteRecordsOnLocalLog(offsetPerPartition: Map[TopicPartition, Long]): Map[TopicPartition, LogDeleteRecordsResult] = { | ||
private def deleteRecordsOnLocalLog(offsetPerPartition: Map[TopicPartition, Long], allowInternalTopicDeletion: Boolean): Map[TopicPartition, LogDeleteRecordsResult] = { | ||
trace("Delete records on local logs to offsets [%s]".format(offsetPerPartition)) | ||
offsetPerPartition.map { case (topicPartition, requestedOffset) => | ||
// reject delete records operation on internal topics | ||
if (Topic.isInternal(topicPartition.topic)) { | ||
// reject delete records operation for internal topics unless allowInternalTopicDeletion is true | ||
if (Topic.isInternal(topicPartition.topic) && !allowInternalTopicDeletion) { | ||
(topicPartition, LogDeleteRecordsResult(-1L, -1L, Some(new InvalidTopicException(s"Cannot delete records of internal topic ${topicPartition.topic}")))) | ||
} else { | ||
try { | ||
|
@@ -1369,9 +1369,10 @@ class ReplicaManager(val config: KafkaConfig, | |
|
||
def deleteRecords(timeout: Long, | ||
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. Let's add a unit test for this new flag in ReplicaManagerTest. |
||
offsetPerPartition: Map[TopicPartition, Long], | ||
responseCallback: Map[TopicPartition, DeleteRecordsPartitionResult] => Unit): Unit = { | ||
responseCallback: Map[TopicPartition, DeleteRecordsPartitionResult] => Unit, | ||
allowInternalTopicDeletion: Boolean = false): Unit = { | ||
val timeBeforeLocalDeleteRecords = time.milliseconds | ||
val localDeleteRecordsResults = deleteRecordsOnLocalLog(offsetPerPartition) | ||
val localDeleteRecordsResults = deleteRecordsOnLocalLog(offsetPerPartition, allowInternalTopicDeletion) | ||
debug("Delete records on local log in %d ms".format(time.milliseconds - timeBeforeLocalDeleteRecords)) | ||
|
||
val deleteRecordsStatus = localDeleteRecordsResults.map { case (topicPartition, result) => | ||
|
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.{assertDoesNotThrow, assertEquals, assertThrows} | ||
import org.junit.jupiter.api.Test | ||
import org.junit.jupiter.params.ParameterizedTest | ||
import org.junit.jupiter.params.provider.EnumSource | ||
|
@@ -238,4 +239,102 @@ class CoordinatorPartitionWriterTest { | |
batch | ||
)) | ||
} | ||
|
||
@Test | ||
def testDeleteRecordsResponseMissingTopicPartition(): 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 does not contain topic partition supplied | ||
when(replicaManager.deleteRecords( | ||
ArgumentMatchers.eq(0L), | ||
ArgumentMatchers.any(), | ||
callbackCapture.capture(), | ||
ArgumentMatchers.eq(true) | ||
)).thenAnswer(_ => { | ||
callbackCapture.getValue.apply(Map( | ||
new TopicPartition("other-topic", 0) -> new DeleteRecordsPartitionResult() | ||
)) | ||
}) | ||
|
||
assertThrows( | ||
classOf[IllegalStateException], | ||
() => partitionRecordWriter.deleteRecords( | ||
new TopicPartition("random-topic", 0), | ||
10L, | ||
allowInternalTopicDeletion = true | ||
) | ||
) | ||
} | ||
|
||
@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.eq(0L), | ||
ArgumentMatchers.any(), | ||
callbackCapture.capture(), | ||
ArgumentMatchers.eq(true) | ||
)).thenAnswer(_ => { | ||
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. |
||
callbackCapture.getValue.apply(Map( | ||
new TopicPartition("random-topic", 0) -> new DeleteRecordsPartitionResult() | ||
.setErrorCode(Errors.NOT_LEADER_OR_FOLLOWER.code()) | ||
)) | ||
}) | ||
|
||
assertThrows( | ||
Errors.NOT_LEADER_OR_FOLLOWER.exception().getClass, | ||
() => partitionRecordWriter.deleteRecords( | ||
new TopicPartition("random-topic", 0), | ||
10L, | ||
allowInternalTopicDeletion = true | ||
) | ||
) | ||
} | ||
|
||
@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.eq(0L), | ||
ArgumentMatchers.any(), | ||
callbackCapture.capture(), | ||
ArgumentMatchers.eq(true) | ||
)).thenAnswer(_ => { | ||
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. |
||
callbackCapture.getValue.apply(Map( | ||
new TopicPartition("random-topic", 0) -> new DeleteRecordsPartitionResult() | ||
.setErrorCode(Errors.NONE.code()) | ||
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. |
||
)) | ||
}) | ||
|
||
assertDoesNotThrow(() => { | ||
partitionRecordWriter.deleteRecords( | ||
new TopicPartition("random-topic", 0), | ||
10L, | ||
allowInternalTopicDeletion = true | ||
) | ||
true | ||
}) | ||
} | ||
} |
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.
This parameter is missing from the javadoc.