[ 
https://issues.apache.org/jira/browse/KAFKA-1806?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14251940#comment-14251940
 ] 

lokesh Birla commented on KAFKA-1806:
-------------------------------------

Hi Neha,

What is the status of fixing this issue? This issue happens on every run. I 
have seen, if I use: num.replica.fetchers=1, then sometimes this issue goes 
away however I see other problem of leadership changes very often even when all 
brokers are running. 

If I set: num.replica.fetchers=4, then I can reproduce this issue on every run. 

Please let me or Evan (from sarama) know if you need any help to fix this. 

> broker can still expose uncommitted data to a consumer
> ------------------------------------------------------
>
>                 Key: KAFKA-1806
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1806
>             Project: Kafka
>          Issue Type: Bug
>          Components: consumer
>    Affects Versions: 0.8.1.1
>            Reporter: lokesh Birla
>            Assignee: Neha Narkhede
>
> Although following issue: https://issues.apache.org/jira/browse/KAFKA-727
> is marked fixed but I still see this issue in 0.8.1.1. I am able to 
> reproducer the issue consistently. 
> [2014-08-18 06:43:58,356] ERROR [KafkaApi-1] Error when processing fetch 
> request for partition [mmetopic4,2] offset 1940029 from consumer with 
> correlation id 21 (kafka.server.Kaf
> kaApis)
> java.lang.IllegalArgumentException: Attempt to read with a maximum offset 
> (1818353) less than the start offset (1940029).
>         at kafka.log.LogSegment.read(LogSegment.scala:136)
>         at kafka.log.Log.read(Log.scala:386)
>         at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$readMessageSet(KafkaApis.scala:530)
>         at 
> kafka.server.KafkaApis$$anonfun$kafka$server$KafkaApis$$readMessageSets$1.apply(KafkaApis.scala:476)
>         at 
> kafka.server.KafkaApis$$anonfun$kafka$server$KafkaApis$$readMessageSets$1.apply(KafkaApis.scala:471)
>         at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:233)
>         at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:233)
>         at scala.collection.immutable.Map$Map1.foreach(Map.scala:119)
>         at 
> scala.collection.TraversableLike$class.map(TraversableLike.scala:233)
>         at scala.collection.immutable.Map$Map1.map(Map.scala:107)
>         at 
> kafka.server.KafkaApis.kafka$server$KafkaApis$$readMessageSets(KafkaApis.scala:471)
>         at 
> kafka.server.KafkaApis$FetchRequestPurgatory.expire(KafkaApis.scala:783)
>         at 
> kafka.server.KafkaApis$FetchRequestPurgatory.expire(KafkaApis.scala:765)
>         at 
> kafka.server.RequestPurgatory$ExpiredRequestReaper.run(RequestPurgatory.scala:216)
>         at java.lang.Thread.run(Thread.java:745)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to