16
16
*/
17
17
package kafka .cluster
18
18
19
+ import java .lang .{Long => JLong }
19
20
import java .util .concurrent .locks .ReentrantReadWriteLock
20
21
import java .util .Optional
21
22
import java .util .concurrent .{CompletableFuture , CopyOnWriteArrayList }
@@ -39,7 +40,7 @@ import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED
39
40
import org .apache .kafka .common .utils .Time
40
41
import org .apache .kafka .metadata .{LeaderAndIsr , LeaderRecoveryState }
41
42
import org .apache .kafka .server .common .RequestLocal
42
- import org .apache .kafka .storage .internals .log .{AppendOrigin , AsyncOffsetReader , FetchDataInfo , LeaderHwChange , LogAppendInfo , LogOffsetMetadata , LogOffsetSnapshot , LogOffsetsListener , LogReadInfo , LogStartOffsetIncrementReason , OffsetResultHolder , VerificationGuard }
43
+ import org .apache .kafka .storage .internals .log .{AppendOrigin , AsyncOffsetReader , FetchDataInfo , LeaderHwChange , LogAppendInfo , LogOffsetMetadata , LogOffsetSnapshot , LogOffsetsListener , LogReadInfo , LogStartOffsetIncrementReason , OffsetResultHolder , UnifiedLog , VerificationGuard }
43
44
import org .apache .kafka .server .metrics .KafkaMetricsGroup
44
45
import org .apache .kafka .server .purgatory .{DelayedOperationPurgatory , TopicPartitionOperationKey }
45
46
import org .apache .kafka .server .share .fetch .DelayedShareFetchPartitionKey
@@ -49,7 +50,7 @@ import org.slf4j.event.Level
49
50
50
51
import scala .collection .Seq
51
52
import scala .jdk .CollectionConverters ._
52
- import scala .jdk .OptionConverters .RichOption
53
+ import scala .jdk .OptionConverters .{ RichOption , RichOptional }
53
54
import scala .jdk .javaapi .OptionConverters
54
55
55
56
/**
@@ -494,7 +495,7 @@ class Partition(val topicPartition: TopicPartition,
494
495
logManager.initializingLog(topicPartition)
495
496
var maybeLog : Option [UnifiedLog ] = None
496
497
try {
497
- val log = logManager.getOrCreateLog(topicPartition, isNew, isFutureReplica, topicId, targetLogDirectoryId)
498
+ val log = logManager.getOrCreateLog(topicPartition, isNew, isFutureReplica, topicId.toJava , targetLogDirectoryId)
498
499
if (! isFutureReplica) log.setLogOffsetsListener(logOffsetsListener)
499
500
maybeLog = Some (log)
500
501
updateHighWatermark(log)
@@ -593,7 +594,7 @@ class Partition(val topicPartition: TopicPartition,
593
594
*/
594
595
def topicId : Option [Uuid ] = {
595
596
if (_topicId.isEmpty || _topicId.contains(Uuid .ZERO_UUID )) {
596
- _topicId = this .log.orElse(logManager.getLog(topicPartition)).flatMap(_.topicId)
597
+ _topicId = this .log.orElse(logManager.getLog(topicPartition)).flatMap(_.topicId.toScala )
597
598
}
598
599
_topicId
599
600
}
@@ -1170,7 +1171,7 @@ class Partition(val topicPartition: TopicPartition,
1170
1171
}
1171
1172
}
1172
1173
1173
- leaderLog.maybeIncrementHighWatermark(newHighWatermark) match {
1174
+ leaderLog.maybeIncrementHighWatermark(newHighWatermark).toScala match {
1174
1175
case Some (oldHighWatermark) =>
1175
1176
debug(s " High watermark updated from $oldHighWatermark to $newHighWatermark" )
1176
1177
true
@@ -1369,8 +1370,7 @@ class Partition(val topicPartition: TopicPartition,
1369
1370
s " live replica(s) broker.id are : $inSyncReplicaIds" )
1370
1371
}
1371
1372
1372
- val info = leaderLog.appendAsLeader(records, leaderEpoch = this .leaderEpoch, origin,
1373
- requestLocal, verificationGuard)
1373
+ val info = leaderLog.appendAsLeader(records, this .leaderEpoch, origin, requestLocal, verificationGuard)
1374
1374
1375
1375
// we may need to increment high watermark since ISR could be down to 1
1376
1376
(info, maybeIncrementLeaderHW(leaderLog))
@@ -1622,7 +1622,7 @@ class Partition(val topicPartition: TopicPartition,
1622
1622
case Some (producers) =>
1623
1623
producerState
1624
1624
.setErrorCode(Errors .NONE .code)
1625
- .setActiveProducers(producers.asJava )
1625
+ .setActiveProducers(producers)
1626
1626
case None =>
1627
1627
producerState
1628
1628
.setErrorCode(Errors .NOT_LEADER_OR_FOLLOWER .code)
@@ -1696,7 +1696,7 @@ class Partition(val topicPartition: TopicPartition,
1696
1696
*/
1697
1697
def truncateFullyAndStartAt (newOffset : Long ,
1698
1698
isFuture : Boolean ,
1699
- logStartOffsetOpt : Option [ Long ] = None ): Unit = {
1699
+ logStartOffsetOpt : Optional [ JLong ] = Optional .empty ): Unit = {
1700
1700
// The read lock is needed to prevent the follower replica from being truncated while ReplicaAlterDirThread
1701
1701
// is executing maybeReplaceCurrentWithFutureReplica() to replace follower replica with the future replica.
1702
1702
inReadLock(leaderIsrUpdateLock) {
@@ -1724,7 +1724,7 @@ class Partition(val topicPartition: TopicPartition,
1724
1724
val localLogOrError = getLocalLog(currentLeaderEpoch, fetchOnlyFromLeader)
1725
1725
localLogOrError match {
1726
1726
case Left (localLog) =>
1727
- localLog.endOffsetForEpoch(leaderEpoch) match {
1727
+ localLog.endOffsetForEpoch(leaderEpoch).toScala match {
1728
1728
case Some (epochAndOffset) => new EpochEndOffset ()
1729
1729
.setPartition(partitionId)
1730
1730
.setErrorCode(Errors .NONE .code)
0 commit comments