@@ -19,13 +19,14 @@ package kafka.server
19
19
import com .yammer .metrics .core .Meter
20
20
import kafka .cluster .{Partition , PartitionListener }
21
21
import kafka .controller .StateChangeLogger
22
- import kafka .log .remote .RemoteLogManager
23
22
import kafka .log .LogManager
23
+ import kafka .log .remote .RemoteLogManager
24
24
import kafka .server .HostedPartition .Online
25
25
import kafka .server .QuotaFactory .QuotaManagers
26
26
import kafka .server .ReplicaManager .{AtMinIsrPartitionCountMetricName , FailedIsrUpdatesPerSecMetricName , IsrExpandsPerSecMetricName , IsrShrinksPerSecMetricName , LeaderCountMetricName , OfflineReplicaCountMetricName , PartitionCountMetricName , PartitionsWithLateTransactionsCountMetricName , ProducerIdCountMetricName , ReassigningPartitionsMetricName , UnderMinIsrPartitionCountMetricName , UnderReplicatedPartitionsMetricName , createLogReadResult , isListOffsetsTimestampUnsupported }
27
27
import kafka .server .share .DelayedShareFetch
28
28
import kafka .utils ._
29
+ import org .apache .kafka .common .{IsolationLevel , Node , TopicIdPartition , TopicPartition , Uuid }
29
30
import org .apache .kafka .common .errors ._
30
31
import org .apache .kafka .common .internals .{Plugin , Topic }
31
32
import org .apache .kafka .common .message .DeleteRecordsResponseData .DeleteRecordsPartitionResult
@@ -46,21 +47,19 @@ import org.apache.kafka.common.requests.FetchRequest.PartitionData
46
47
import org .apache .kafka .common .requests .ProduceResponse .PartitionResponse
47
48
import org .apache .kafka .common .requests ._
48
49
import org .apache .kafka .common .utils .{Exit , Time , Utils }
49
- import org .apache .kafka .common .{IsolationLevel , Node , TopicIdPartition , TopicPartition , Uuid }
50
50
import org .apache .kafka .image .{LocalReplicaChanges , MetadataImage , TopicsDelta }
51
51
import org .apache .kafka .metadata .LeaderConstants .NO_LEADER
52
52
import org .apache .kafka .metadata .MetadataCache
53
- import org .apache .kafka .server .{ActionQueue , DelayedActionQueue , ListOffsetsPartitionStatus , common }
54
53
import org .apache .kafka .server .common .{DirectoryEventHandler , RequestLocal , StopPartition , TopicOptionalIdPartition }
55
54
import org .apache .kafka .server .metrics .KafkaMetricsGroup
56
55
import org .apache .kafka .server .network .BrokerEndPoint
57
- import org .apache .kafka .server .purgatory .{DelayedOperationPurgatory , TopicPartitionOperationKey }
56
+ import org .apache .kafka .server .purgatory .{DelayedDeleteRecords , DelayedOperationPurgatory , DeleteRecordsPartitionStatus , TopicPartitionOperationKey }
58
57
import org .apache .kafka .server .share .fetch .{DelayedShareFetchKey , DelayedShareFetchPartitionKey }
59
58
import org .apache .kafka .server .storage .log .{FetchParams , FetchPartitionData }
60
59
import org .apache .kafka .server .util .{Scheduler , ShutdownableThread }
60
+ import org .apache .kafka .server .{ActionQueue , DelayedActionQueue , ListOffsetsPartitionStatus , common }
61
61
import org .apache .kafka .storage .internals .checkpoint .{LazyOffsetCheckpoints , OffsetCheckpointFile , OffsetCheckpoints }
62
- import org .apache .kafka .storage .internals .log .{AppendOrigin , FetchDataInfo , LeaderHwChange , LogAppendInfo , LogConfig , LogDirFailureChannel , LogOffsetMetadata , LogReadInfo , OffsetResultHolder , RecordValidationException , RemoteLogReadResult , RemoteStorageFetchInfo , UnifiedLog => UnifiedLog , VerificationGuard }
63
-
62
+ import org .apache .kafka .storage .internals .log ._
64
63
import org .apache .kafka .storage .log .metrics .BrokerTopicStats
65
64
66
65
import java .io .File
@@ -1314,7 +1313,7 @@ class ReplicaManager(val config: KafkaConfig,
1314
1313
1315
1314
val deleteRecordsStatus = localDeleteRecordsResults.map { case (topicPartition, result) =>
1316
1315
topicPartition ->
1317
- DeleteRecordsPartitionStatus (
1316
+ new DeleteRecordsPartitionStatus (
1318
1317
result.requestedOffset, // requested offset
1319
1318
new DeleteRecordsPartitionResult ()
1320
1319
.setLowWatermark(result.lowWatermark)
@@ -1323,8 +1322,31 @@ class ReplicaManager(val config: KafkaConfig,
1323
1322
}
1324
1323
1325
1324
if (delayedDeleteRecordsRequired(localDeleteRecordsResults)) {
1325
+ def onAcks (topicPartition : TopicPartition , status : DeleteRecordsPartitionStatus ): Unit = {
1326
+ val (lowWatermarkReached, error, lw) = getPartition(topicPartition) match {
1327
+ case HostedPartition .Online (partition) =>
1328
+ partition.leaderLogIfLocal match {
1329
+ case Some (_) =>
1330
+ val leaderLW = partition.lowWatermarkIfLeader
1331
+ (leaderLW >= status.requiredOffset, Errors .NONE , leaderLW)
1332
+ case None =>
1333
+ (false , Errors .NOT_LEADER_OR_FOLLOWER , DeleteRecordsResponse .INVALID_LOW_WATERMARK )
1334
+ }
1335
+
1336
+ case HostedPartition .Offline (_) =>
1337
+ (false , Errors .KAFKA_STORAGE_ERROR , DeleteRecordsResponse .INVALID_LOW_WATERMARK )
1338
+
1339
+ case HostedPartition .None =>
1340
+ (false , Errors .UNKNOWN_TOPIC_OR_PARTITION , DeleteRecordsResponse .INVALID_LOW_WATERMARK )
1341
+ }
1342
+ if (error != Errors .NONE || lowWatermarkReached) {
1343
+ status.setAcksPending(false )
1344
+ status.responseStatus.setErrorCode(error.code)
1345
+ status.responseStatus.setLowWatermark(lw)
1346
+ }
1347
+ }
1326
1348
// create delayed delete records operation
1327
- val delayedDeleteRecords = new DelayedDeleteRecords (timeout, deleteRecordsStatus, this , responseCallback)
1349
+ val delayedDeleteRecords = new DelayedDeleteRecords (timeout, deleteRecordsStatus.asJava, onAcks, response => responseCallback(response.asScala) )
1328
1350
1329
1351
// create a list of (topic, partition) pairs to use as keys for this delayed delete records operation
1330
1352
val deleteRecordsRequestKeys = offsetPerPartition.keys.map(new TopicPartitionOperationKey (_)).toList
0 commit comments