[ https://issues.apache.org/jira/browse/KAFKA-860?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13628674#comment-13628674 ]
Jun Rao commented on KAFKA-860: ------------------------------- Thanks for patch v2. This turns out to be a bit more tricky. 1. First of all, instead of using "leaderIsrReadLock synchronized", we should do "leaderIsrReadLock.lock()". 2. Second, we should use a fair readWriteLock. Otherwise, some threads may be indefinitely postponed. 3. Third, from java doc, ReentrantReadWriteLock doesn't support upgrading from read lock to write loc. " Lock downgrading Reentrancy also allows downgrading from the write lock to a read lock, by acquiring the write lock, then the read lock and then releasing the write lock. However, upgrading from a read lock to the write lock is not possible. " This means that if we need to call updateIsr(), we have to first release the read lock and require the read lock again when done. See the following example. However, this means that we are still vulnerable to the issue in maybeIncrementLeaderHW() (kafka-862). We probably can change the logic in maybeIncrementLeaderHW() so that it can handle empty set. We will need to think a bit more how to write the logic in a clean way. http://codereview.stackexchange.com/questions/12939/reentrantreadwritelock-lock-upgrade-method Another possibility is to just take v1 patch. All producers to the same log will sync on the leaderIsrUpdateLock. In log.append(), the only code outside the log lock are analyzeAndValidateMessageSet() and maybeFlush(). The former is cheap since it does shallow iteration. The latter re-requires the log lock if flush if needed. > Replica fetcher thread errors out and dies during rolling bounce of cluster > --------------------------------------------------------------------------- > > Key: KAFKA-860 > URL: https://issues.apache.org/jira/browse/KAFKA-860 > Project: Kafka > Issue Type: Bug > Components: replication > Affects Versions: 0.8 > Reporter: Neha Narkhede > Assignee: Neha Narkhede > Priority: Blocker > Labels: kafka-0.8, p1 > Attachments: kafka-860-v1.patch, kafka-860-v2.patch > > > 2013/04/10 20:04:32.071 ERROR [ReplicaFetcherThread] > [ReplicaFetcherThread-0-272] [kafka] [] [ReplicaFetcherThread-0-272], Error > due to > kafka.common.KafkaException: error processing data for topic PageViewEvent > partititon 3 offset 2482625623 > at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$4.apply(AbstractFetcherThread.scala:135) > at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$4.apply(AbstractFetcherThread.scala:113) > at scala.collection.immutable.Map$Map1.foreach(Map.scala:105) > at > kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:113) > at > kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:89) > at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:51) > Caused by: java.lang.RuntimeException: Offset mismatch: fetched offset = > 2482625623, log end offset = 2482625631. > at > kafka.server.ReplicaFetcherThread.processPartitionData(ReplicaFetcherThread.scala:49) > at > kafka.server.AbstractFetcherThread$$anonfun$processFetchRequest$4.apply(AbstractFetcherThread.scala:132) > ... 5 more > This causes replica fetcher thread to shut down -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira