@ -131,6 +131,8 @@ class Partition(val topic: String,
@@ -131,6 +131,8 @@ class Partition(val topic: String,
leaderEpoch = leaderAndISR . leaderEpoch
zkVersion = leaderAndISR . zkVersion
leaderReplicaIdOpt = Some ( localBrokerId )
// we may need to increment high watermark since ISR could be down to 1
maybeIncrementLeaderHW ( getReplica ( ) . get )
}
/* *
@ -210,7 +212,7 @@ class Partition(val topic: String,
@@ -210,7 +212,7 @@ class Partition(val topic: String,
}
}
private def maybeIncrementLeaderHW ( leaderReplica : Replica ) {
def maybeIncrementLeaderHW ( leaderReplica : Replica ) {
val allLogEndOffsets = inSyncReplicas . map ( _ . logEndOffset )
val newHighWatermark = allLogEndOffsets . min
val oldHighWatermark = leaderReplica . highWatermark
@ -232,6 +234,8 @@ class Partition(val topic: String,
@@ -232,6 +234,8 @@ class Partition(val topic: String,
info ( "Shrinking ISR for topic %s partition %d to %s" . format ( topic , partitionId , newInSyncReplicas . map ( _ . brokerId ) . mkString ( "," ) ) )
// update ISR in zk and in cache
updateISR ( newInSyncReplicas )
// we may need to increment high watermark since ISR could be down to 1
maybeIncrementLeaderHW ( leaderReplica )
}
case None => // do nothing if no longer leader
}