You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Jun Rao (JIRA)" <ji...@apache.org> on 2012/09/24 18:52:07 UTC

[jira] [Commented] (KAFKA-509) server should shut down on encountering invalid highwatermark file

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

Jun Rao commented on KAFKA-509:
-------------------------------

The first 2 lines are valid and they indicate version # of # of partitions. The \0 could just be a wrong topic. In any case, we should deal with potential special characters (e.g., space) in the topic name. One way to do this is to use '/' as the delimiter since it's disallowed in topic name. Also, if we hit any exception while reading high watermark, we should log the problematic content.

Finally, I recommend that we take care of the following comments left from kafka-405.
40. HighwaterMarkCheckpoint: 
40.1 If tempHwFile already exists, we can just overwrite it since we know hwFile is always safe. 
40.2 There is no need to delete hwFile first and then rename tempHwFile to it. Rename should do the deletion. Currently, if we fail at the bad time, we could end up without a hwFile.
                
> server should shut down on encountering invalid highwatermark file
> ------------------------------------------------------------------
>
>                 Key: KAFKA-509
>                 URL: https://issues.apache.org/jira/browse/KAFKA-509
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.8
>            Reporter: Swapnil Ghike
>            Priority: Blocker
>              Labels: bugs
>             Fix For: 0.8
>
>
> 1. Somehow I managed to produce the following .highwatermark file (most probably while playing with kafka-create-topic.sh) - 
> 0
> 7
> abra.kabra 0 0
> 0 0 0
> abrakabra 0 0
> \0 0 0
> king. 0 0
> abra..kabra 0 0
> ... 0 0
> Perhaps the first two lines are not valid. But I am not able to reproduce this issue today. 
> 2. Since the topic names can contain a whitespace, perhaps the delimiter should change from a space char to / which is not allowed anymore in topic names.
> 3. With this .highwatermark file, the kafka server produces following - 
> [2012-09-12 14:54:49,456] INFO Replica Manager on Broker 0: Becoming Leader for topic [abra.kabra] partition [0] (kafka.server.ReplicaManager)
> [2012-09-12 14:54:49,456] INFO [ReplicaFetcherManager on broker 0, ], removing fetcher on topic abra.kabra, partition 0 (kafka.server.ReplicaFetcherManager)
> [2012-09-12 14:54:49,457] ERROR Replica Manager on Broker 0: Error processing leaderAndISR request LeaderAndIsrRequest(1,,true,1000,Map((...,0) -> { "ISR": "0","leader": "0","leaderEpoch": "0" }, (\0,0) -> { "ISR": "0","leader": "0","leaderEpoch": "0" }, (abrakabra,0) -> { "ISR": "0","leader": "0","leaderEpoch": "0" }, (0,0) -> { "ISR": "0","leader": "0","leaderEpoch": "0" }, (abra..kabra,0) -> { "ISR": "0","leader": "0","leaderEpoch": "0" }, (king.,0) -> { "ISR": "0","leader": "0","leaderEpoch": "0" }, (abra.kabra,0) -> { "ISR": "0","leader": "0","leaderEpoch": "0" })) (kafka.server.ReplicaManager)
> java.lang.StringIndexOutOfBoundsException: String index out of range: -1
> 	at java.lang.String.substring(String.java:1937)
> 	at kafka.server.HighwaterMarkCheckpoint$$anonfun$1.apply(HighwaterMarkCheckpoint.scala:103)
> 	at kafka.server.HighwaterMarkCheckpoint$$anonfun$1.apply(HighwaterMarkCheckpoint.scala:96)
> 	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:206)
> 	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:206)
> 	at scala.collection.immutable.Range$ByOne$class.foreach(Range.scala:282)
> 	at scala.collection.immutable.Range$$anon$2.foreach(Range.scala:265)
> 	at scala.collection.TraversableLike$class.map(TraversableLike.scala:206)
> 	at scala.collection.immutable.Range.map(Range.scala:39)
> 	at kafka.server.HighwaterMarkCheckpoint.read(HighwaterMarkCheckpoint.scala:96)
>         ... (more junk) ..
> And then it goes in a while(1) loop to print the following - 
> [2012-09-12 14:54:50,458] ERROR Replica Manager on Broker 0: Highwatermark for topic abra.kabra partition 0 doesn't exist during checkpointing (kafka.server.ReplicaManager)
> [2012-09-12 14:54:50,459] ERROR Replica Manager on Broker 0: Highwatermark for topic ... partition 0 doesn't exist during checkpointing (kafka.server.ReplicaManager)
> [2012-09-12 14:54:50,459] ERROR Replica Manager on Broker 0: Highwatermark for topic \0 partition 0 doesn't exist during checkpointing (kafka.server.ReplicaManager)
> [2012-09-12 14:54:50,459] ERROR Replica Manager on Broker 0: Highwatermark for topic 0 partition 0 doesn't exist during checkpointing (kafka.server.ReplicaManager)
> [2012-09-12 14:54:50,459] ERROR Replica Manager on Broker 0: Highwatermark for topic king. partition 0 doesn't exist during checkpointing (kafka.server.ReplicaManager)
> [2012-09-12 14:54:50,460] ERROR Replica Manager on Broker 0: Highwatermark for topic abra..kabra partition 0 doesn't exist during checkpointing (kafka.server.ReplicaManager)
> [2012-09-12 14:54:50,460] ERROR Replica Manager on Broker 0: Highwatermark for topic abrakabra partition 0 doesn't exist during checkpointing (kafka.server.ReplicaManager)
> The server should shut down on encountering the error.

--
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