You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Joel Koshy (JIRA)" <ji...@apache.org> on 2014/04/16 03:35:14 UTC

[jira] [Created] (KAFKA-1398) Topic config changes can be lost and cause fatal exceptions on broker restarts

Joel Koshy created KAFKA-1398:
---------------------------------

             Summary: Topic config changes can be lost and cause fatal exceptions on broker restarts
                 Key: KAFKA-1398
                 URL: https://issues.apache.org/jira/browse/KAFKA-1398
             Project: Kafka
          Issue Type: Bug
    Affects Versions: 0.8.1
            Reporter: Joel Koshy
            Priority: Critical
             Fix For: 0.8.1.1


Our topic config cleanup policy seems to be broken. When a broker is
bounced and starting up:
1 - Read all the children of the config change path
2 - For each, if the change id is greater than the last executed change,
  then extract the topic information.
3 - If there is a log for that topic on this broker, then apply the change.
  However, if there is no log, then delete the config change.

In step 3, a delete triggers a child change watch firing on all the other
brokers. The other brokers currently take all the children of the config
path but will ignore those config changes that are less than the last
executed change. At least one issue here is that if a broker does not have
partitions for a topic then the lastExecutedChange is not updated (for
that topic).

Consider this scenario:
- Three brokers 0, 1, 2
- Topic A has partitions only assigned to broker 0
- Topic B has partitions only assigned to broker 1
- Topic C has partitions only assigned to broker 2
- Change 0: topic A
- Change 1: topic B
- Change 2: topic C
- lastExecutedChange on broker 0 is 0
- lastExecutedChange on broker 1 is 1
- lastExecutedChange on broker 2 is 2
- Bounce broker 1
- The above bounce will cause Change 0 and Change 2 to get deleted.
- Watch fires on broker 0 and 1
- Broker 0 will try and read the topic corresponding to change 1 (since its
  lastExecutedChange is 0) and then change 2. That read will fail:

2014/04/15 19:35:34.236 INFO [TopicConfigManager] [main] [kafka-server] [] Processed topic config change 25 for topic xyz, setting new config to
 {retention.ms=3600000, segment.ms=3600000}.
2014/04/15 19:35:34.238 FATAL [KafkaServerStartable] [main] [kafka-server] [] Fatal error during KafkaServerStable startup. Prepare to shutdown
org.I0Itec.zkclient.exception.ZkNoNodeException: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /config/changes/config_change_0000000026
        at org.I0Itec.zkclient.exception.ZkException.create(ZkException.java:47)
        at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:685)
        at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:766)
        at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:761)
        at kafka.utils.ZkUtils$.readData(ZkUtils.scala:467)
        at kafka.server.TopicConfigManager$$anonfun$kafka$server$TopicConfigManager$$processConfigChanges$2.apply(TopicConfigManager.scala:97)
        at kafka.server.TopicConfigManager$$anonfun$kafka$server$TopicConfigManager$$processConfigChanges$2.apply(TopicConfigManager.scala:93)
        at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:57)
        at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:43)
        at kafka.server.TopicConfigManager.kafka$server$TopicConfigManager$$processConfigChanges(TopicConfigManager.scala:93)
        at kafka.server.TopicConfigManager.processAllConfigChanges(TopicConfigManager.scala:81)
        at kafka.server.TopicConfigManager.startup(TopicConfigManager.scala:72)
        at kafka.server.KafkaServer.startup(KafkaServer.scala:104)
        at kafka.server.KafkaServerStartable.startup(KafkaServerStartable.scala:34)
        ...
Caused by: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /config/changes/config_change_0000000026
        at org.apache.zookeeper.KeeperException.create(KeeperException.java:102)
        at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
        at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:927)
        at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:956)
        at org.I0Itec.zkclient.ZkConnection.readData(ZkConnection.java:103)
        at org.I0Itec.zkclient.ZkClient$9.call(ZkClient.java:770)
        at org.I0Itec.zkclient.ZkClient$9.call(ZkClient.java:766)
        at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
        ... 39 more


Another issue is that there are two logging statements with incorrect
qualifiers which makes things a little harder to debug. E.g.,

2014/04/15 19:35:34.223 ERROR [TopicConfigManager] [kafka-server] [] Ignoring topic config change %d for topic %s since the change has expired





--
This message was sent by Atlassian JIRA
(v6.2#6252)