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

ASF GitHub Bot commented on KAFKA-4196:
---------------------------------------

GitHub user original-brownbear opened a pull request:

    https://github.com/apache/kafka/pull/2570

    KAFKA-4196: Improved Test Stability by Disabling ZK Fsync and Fixed 
KafkaAPI Error Response

    This addresses https://issues.apache.org/jira/browse/KAFKA-4196
    
    What I found was below warning accompanying all failures I was seeing from 
this test (reproduced instability by putting system under load):
    
    ```sh
    [2017-02-18 16:17:42,892] WARN fsync-ing the write ahead log in 
SyncThread:0 took 20632ms which will adversely effect operation latency. See 
the ZooKeeper troubleshooting guide 
(org.apache.zookeeper.server.persistence.FileTxnLog:338)
    ```
    
    ZK at times keeps locking for multiple seconds in tests (not only this one, 
but it's very frequent in this one for some reason). In this case (20s) the ZK 
locking lasted longer than the test timeout waiting only 15s 
(`org.apache.kafka.test.TestUtils#DEFAULT_MAX_WAIT_MS`) for the path 
`/admin/delete_topic/topic` to be deleted.
    The only way to really fix this in a portable manner (should mainly hit 
ext3 users) is to turn off ZK fsyncing (not really needed in UTs anyways) as 
far as I know.
    Did that here as described in 
(https://zookeeper.apache.org/doc/trunk/zookeeperAdmin.html) by setting
    
    ```scala
      sys.props.put("zookeeper.observer.syncEnabled", "false")
    ```
    
    This should also help general test performance in my opinion.
    
    Also fixed (only ever observed this here) that the resulting error was not 
properly logged by the `KafkaApis` since no type param was given in the changed 
line
    
    ```scala
              error("Error when handling request %s".format(request.body), e)
    ```
    
    that then threw:
    
    ```sh
    java.lang.ClassCastException: Expected request with type class 
scala.runtime.Nothing$, but found class 
org.apache.kafka.common.requests.UpdateMetadataRequest
        at kafka.network.RequestChannel$Request.body(RequestChannel.scala:118)
        at kafka.server.KafkaApis$$anonfun$handle$4.apply(KafkaApis.scala:120)
        at kafka.server.KafkaApis$$anonfun$handle$4.apply(KafkaApis.scala:120)
        at kafka.utils.Logging$class.error(Logging.scala:105)
        at kafka.server.KafkaApis.error(KafkaApis.scala:56)
        at kafka.server.KafkaApis.handle(KafkaApis.scala:120)
        at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:64)
        at java.lang.Thread.run(Thread.java:745)
    ```
    
    added the hint there and (without the fsync fix) got logged proper errors :)

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/original-brownbear/kafka KAFKA-4196

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/kafka/pull/2570.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2570
    
----
commit 99dd9c62f63960bac35effd6a2514cd5ba61d66a
Author: Armin Braun <m...@obrown.io>
Date:   2017-02-18T16:24:31Z

    KAFKA-4196 Improved Test Stability by Disabling ZK Fsync and Fixed KafkaAPI 
Error Response

----


> Transient test failure: 
> DeleteConsumerGroupTest.testConsumptionOnRecreatedTopicAfterTopicWideDeleteInZK
> -------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-4196
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4196
>             Project: Kafka
>          Issue Type: Sub-task
>            Reporter: Ismael Juma
>              Labels: transient-unit-test-failure
>
> The error:
> {code}
> java.lang.AssertionError: Admin path /admin/delete_topic/topic path not 
> deleted even after a replica is restarted
>       at org.junit.Assert.fail(Assert.java:88)
>       at kafka.utils.TestUtils$.waitUntilTrue(TestUtils.scala:752)
>       at kafka.utils.TestUtils$.verifyTopicDeletion(TestUtils.scala:1017)
>       at 
> kafka.admin.DeleteConsumerGroupTest.testConsumptionOnRecreatedTopicAfterTopicWideDeleteInZK(DeleteConsumerGroupTest.scala:156)
> {code}
> Caused by a broken invariant in the Controller: a partition exists in 
> `ControllerContext.partitionLeadershipInfo`, but not 
> `controllerContext.partitionReplicaAssignment`.
> {code}
> [2016-09-20 06:45:13,967] ERROR [BrokerChangeListener on Controller 1]: Error 
> while handling broker changes 
> (kafka.controller.ReplicaStateMachine$BrokerChangeListener:103)
> java.util.NoSuchElementException: key not found: [topic,0]
>       at scala.collection.MapLike$class.default(MapLike.scala:228)
>       at scala.collection.AbstractMap.default(Map.scala:58)
>       at scala.collection.mutable.HashMap.apply(HashMap.scala:64)
>       at 
> kafka.controller.ControllerBrokerRequestBatch.kafka$controller$ControllerBrokerRequestBatch$$updateMetadataRequestMapFor$1(ControllerChannelManager.scala:310)
>       at 
> kafka.controller.ControllerBrokerRequestBatch$$anonfun$addUpdateMetadataRequestForBrokers$4.apply(ControllerChannelManager.scala:343)
>       at 
> kafka.controller.ControllerBrokerRequestBatch$$anonfun$addUpdateMetadataRequestForBrokers$4.apply(ControllerChannelManager.scala:343)
>       at scala.collection.mutable.HashSet.foreach(HashSet.scala:79)
>       at 
> kafka.controller.ControllerBrokerRequestBatch.addUpdateMetadataRequestForBrokers(ControllerChannelManager.scala:343)
>       at 
> kafka.controller.KafkaController.sendUpdateMetadataRequest(KafkaController.scala:1030)
>       at 
> kafka.controller.KafkaController.onBrokerFailure(KafkaController.scala:492)
>       at 
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(ReplicaStateMachine.scala:376)
>       at 
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply(ReplicaStateMachine.scala:358)
>       at 
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply(ReplicaStateMachine.scala:358)
>       at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:33)
>       at 
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1.apply$mcV$sp(ReplicaStateMachine.scala:357)
>       at 
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1.apply(ReplicaStateMachine.scala:356)
>       at 
> kafka.controller.ReplicaStateMachine$BrokerChangeListener$$anonfun$handleChildChange$1.apply(ReplicaStateMachine.scala:356)
>       at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:234)
>       at 
> kafka.controller.ReplicaStateMachine$BrokerChangeListener.handleChildChange(ReplicaStateMachine.scala:355)
>       at org.I0Itec.zkclient.ZkClient$10.run(ZkClient.java:843)
>       at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to