-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/15674/#review29118
-----------------------------------------------------------



core/src/main/scala/kafka/log/LogSegment.scala
<https://reviews.apache.org/r/15674/#comment56244>

    Probably we could throw the exception if log.delete failed so we still have 
the index for trouble shooting?


Currently we have the the locking hierarchy as:

ReplicaManager : replicaStateChangeLock
|
Partition : leaderIsrUpdateLock
|
LogManager : logCreationOrDeletionLock
|
Log : this.lock
|
LogSegment : this.lock

I think probably we can reduce some of the lower level synchronization if they 
have already been covered by the higher level ones?

- Guozhang Wang


On Nov. 19, 2013, 4:28 p.m., Jun Rao wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/15674/
> -----------------------------------------------------------
> 
> (Updated Nov. 19, 2013, 4:28 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1074
>     https://issues.apache.org/jira/browse/KAFKA-1074
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> kafka-1074; fix 3
> 
> 
> kafka-1074; fix 2
> 
> 
> kafka-1074
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/cluster/Partition.scala 
> 02ccc17c79b6d44c75f9bb6ca7cda8c51ae6f6fb 
>   core/src/main/scala/kafka/log/Log.scala 
> 1883a53de112ad08449dc73a2ca08208c11a2537 
>   core/src/main/scala/kafka/log/LogManager.scala 
> 81be88aa618ed5614703d45a0556b77c97290085 
>   core/src/main/scala/kafka/log/LogSegment.scala 
> 0d6926ea105a99c9ff2cfc9ea6440f2f2d37bde8 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 
> 161f58134f20f9335dbd2bee6ac3f71897cbef7c 
>   core/src/test/scala/unit/kafka/admin/AdminTest.scala 
> c30069e837e54fb91bf1d5b75b133282a28dedf8 
> 
> Diff: https://reviews.apache.org/r/15674/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Jun Rao
> 
>

Reply via email to