Topic deletion doesn't quite work in 0.8.1.1. It's fixed in the upcoming 0.8.2 release.
Thanks, Jun On Wed, Dec 3, 2014 at 6:17 PM, Andrew Jorgensen < ajorgen...@twitter.com.invalid> wrote: > We are currently running 0.8.1.1, I just double checked. One other thing > that may be related is I brought up a second kafka cluster today matching > the first. I noticed that if I deleted a topic and the re-created it with > the same name when I re-created the topic none of the leader elections > happened and doing a describe on the topic all of the partitions say > “Leader: none”. If I bounce the controller process then the leader election > happens and the partitions get properly assigned. The other interesting > thing that happened was I found that if I kill -9 the MirrorMaker process > while its running it seems like the state cannot be properly resumed when > the MirrorMaker is restarted. Here are what the errors looked like: > > [2014-12-04 02:14:01,025] ERROR > [ConsumerFetcherThread-topic_1__mirror_kafka2009-1417659239211-f603e23f-0-7], > Current offset 817179999 for partition [topic-1,266] out of range; reset > offset to 823942799 (kafka.consumer.ConsumerFetcherThread) > [2014-12-04 02:14:01,028] ERROR > [ConsumerFetcherThread-topic_1_mirror_kafka2009-1417659239211-f603e23f-0-7], > Current offset 941288745 for partition [topic-1,158] out of range; reset > offset to 947524545 (kafka.consumer.ConsumerFetcherThread) > > These errors are remedied by bouncing the controller process and then > restarting the MirrorMaker. To me it looks like there is some disconnect > between the state that exists in zookeeper and what the controller know > about the world. I can reliably repeat both the MirrorMaker and the topic > deletion experiments. If there is anything specific I can help to diagnose > let me know. I can also open a JIRA ticket some these details if it will > help. > > -- > Andrew Jorgensen > @ajorgensen > > On December 3, 2014 at 7:48:20 PM, Jun Rao (jun...@gmail.com) wrote: > > Not sure exactly what happened there. We did fix a few bugs in reassigning > partitions in 0.8.1.1. So, you probably want to upgrade to that one or the > upcoming 0.8.2 release. > > Thanks, > > Jun > > On Tue, Dec 2, 2014 at 2:33 PM, Andrew Jorgensen <ajorgen...@twitter.com> > wrote: > >> I am using kafka 0.8. >> Yes I did run —verify, but got some weird output from it I had never >> seen before that looked something like: >> >> Status of partition reassignment: >> ERROR: Assigned replicas (5,2) don't match the list of replicas for >> reassignment (5) for partition [topic-1,248] >> ERROR: Assigned replicas (7,3) don't match the list of replicas for >> reassignment (7) for partition [topic-2,228] >> >> There were a large number of these but it seems to just be for topic-1, >> and topic-2. In this case I was migrating around 4 or 5 topics. These two >> are also the ones that got reversed when I bounced all the processes >> yesterday. >> >> Here are some more logs that I found from that day that may help piece >> together what might have happened >> >> [2014-11-19 16:56:52,938] ERROR [KafkaApi-1] Error when processing fetch >> request for partition [topic-2,317] offset 408324093 from follower with >> correlation id 2458 (kafka.server.KafkaApis) >> kafka.common.OffsetOutOfRangeException: Request for offset 408324093 but >> we only have log segments in the range 409018400 to 425346400. >> at kafka.log.Log.read(Log.scala:380) >> 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.HashMap$HashMap1.foreach(HashMap.scala:17[image: >> 8)] >> at >> scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:347) >> at >> scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:347) >> at scala.collection.TraversableLike$class.map(TraversableLike.scala:233) >> at scala.collection.immutable.HashMap.map(HashMap.scala:3[image: 8)] >> at >> kafka.server.KafkaApis.kafka$server$KafkaApis$$readMessageSets(KafkaApis.scala:471) >> at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:437) >> at kafka.server.KafkaApis.handle(KafkaApis.scala:186) >> at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:42) >> at java.lang.Thread.run(Thread.java:724) >> >> ----- >> >> [2014-11-19 16:24:37,959] ERROR Conditional update of path >> /brokers/topics/topic-2/partitions/248/state with data >> {"controller_epoch":15,"leader":2,"version":1,"leader_epoch":1,"isr":[2,5]} >> and expected version 1 failed due to >> org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = >> BadVersion for /brokers/topics/topic-2/partitions/248/state >> (kafka.utils.ZkUtils$) >> >> >> >> -- >> Andrew Jorgensen >> @ajorgensen >> >> On December 2, 2014 at 5:28:07 PM, Jun Rao (jun...@gmail.com) wrote: >> >> Did you run the --verify option ( >> http://kafka.apache.org/documentation.html#basic_ops_restarting) to check >> if the reassignment process completes? Also, what version of Kafka are you >> using? >> >> Thanks, >> >> Jun >> >> On Mon, Dec 1, 2014 at 7:16 PM, Andrew Jorgensen < >> ajorgen...@twitter.com.invalid> wrote: >> >> > I unfortunately do not have any specific logs from these events but I >> will >> > try and describe the events as accurately as possible to give an idea >> of >> > the problem I saw. >> > >> > The odd behavior manifested itself when I bounced all of the kafka >> > processes on each of the servers in a 12 node cluster. A few weeks >> prior I >> > did a partition reassignment to add four new kafka brokers to the >> cluster. >> > This cluster has 4 topics on it each with 350 partitions each, a >> retention >> > policy of 6 hours, and a replication factor of 1. Originally I >> attempted to >> > run a migration on all of the topics and partitions adding the 4 new >> nodes >> > using the partition reassignment tool. This seemed to cause a lot of >> > network congestion and according to the logs some of the nodes were >> having >> > trouble talking to each other. The network congestion lasted for the >> > duration of the migration and began to get better toward the end. After >> the >> > migration I confirmed that data was being stored and served from the new >> > brokers. Today I bounced each of the kafka processes on each of the >> brokers >> > to pick up a change made to the log4j properties. After bouncing one >> > processes I started seeing some strange errors on the four newer broker >> > nodes that looked like: >> > >> > kafka.common.NotAssignedReplicaException: Leader 10 failed to record >> > follower 7's position 0 for partition [topic-1,185] since the replica 7 >> is >> > not recognized to be one of the assigned replicas 10 for partition >> > [topic-2,185] >> > >> > and on the older kafka brokers the errors looked like: >> > >> > [2014-12-01 17:06:04,268] ERROR [ReplicaFetcherThread-0-12], Error for >> > partition [topic-1,175] to broker 12:class kafka.common.UnknownException >> > (kafka.server.ReplicaFetcherThread) >> > >> > I proceeded to bounce the rest of the kafka processes and after bouncing >> > the rest the errors seemed to stop. It wasn’t until a few hours later I >> > noticed that the amount of data stored on the 4 new kafka brokers had >> > dropped off significantly. When I ran a describe for the topics in the >> > errors it was clear that the assigned partitions had been reverted to a >> > state prior to the original migration to add the 4 new brokers. I am >> unsure >> > of why bouncing the kafka process would cause the state in zookeeper to >> get >> > overwritten given that it had seemed to have been working for the last >> few >> > weeks until the process was restarted. My hunch is that the controller >> > keeps some state about the world pre-reassignment and removes that state >> > after it detects that the reassignment happened successfully. In this >> case >> > the network congestion on each of the brokers caused the controller not >> to >> > get notified when all the reassignments were completed and thus kept the >> > pre-assignement state around. When the process was bounced it read from >> > zookeeper to get this state and reverted the existing scheme to the >> > pre-assignment state. Has this behavior been observed before? Does this >> > sound like a logical understanding of what happened in this case? >> > >> > -- >> > Andrew Jorgensen >> > @ajorgensen >> >> >