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

Rajini Sivaram commented on KAFKA-2910:
---------------------------------------

Came across this failure while running the tests locally. I think the issue is 
with tests which don't close Zookeeper clients. Unclosed clients continue to 
attempt to reconnect to the Zookeeper server after the server is shutdown, and 
this loads JAAS configuration. Subsequent tests reuse the JAAS configuration, 
resulting in this transient failure.

One possible sequence causing failure:
# TestA creates Zookeeper server in its setUp
# TestA creates ZkUtils in a test
# TestA shuts down Zookeeper server in its tearDown
# TestA calls {{Configuration.setConfiguration(null)}} in its tearDown to reset 
JAAS config
# TestB starts
# TestA's ZK client sender thread which was not closed attempts to reconnect, 
calling {{Configuration.getConfiguration()}}. JAAS config is now reloaded 
because it was reset by TestA's tearDown. At this point the JAAS config loaded 
is typically an empty config.
# TestB creates JAAS config file and sets System property 
{{java.security.auth.login.config}}
# TestB creates Zookeeper and Kafka servers, expecting 
{{Configuration.getConfiguration()}} to load the config based on the currently 
set System property {{java.security.auth.login.config}}. But since the 
configuration was already loaded in  step 6) by TestA before the System 
property was set, JAAS config is not reloaded. TestB setUp fails as a result.

> Failure in kafka.api.SslEndToEndAuthorizationTest.testNoGroupAcl
> ----------------------------------------------------------------
>
>                 Key: KAFKA-2910
>                 URL: https://issues.apache.org/jira/browse/KAFKA-2910
>             Project: Kafka
>          Issue Type: Sub-task
>            Reporter: Guozhang Wang
>            Assignee: Rajini Sivaram
>             Fix For: 0.10.1.0
>
>
> {code}
> java.lang.SecurityException: zkEnableSecureAcls is true, but the verification 
> of the JAAS login file failed.
>       at kafka.server.KafkaServer.initZk(KafkaServer.scala:265)
>       at kafka.server.KafkaServer.startup(KafkaServer.scala:168)
>       at kafka.utils.TestUtils$.createServer(TestUtils.scala:143)
>       at 
> kafka.integration.KafkaServerTestHarness$$anonfun$setUp$1.apply(KafkaServerTestHarness.scala:66)
>       at 
> kafka.integration.KafkaServerTestHarness$$anonfun$setUp$1.apply(KafkaServerTestHarness.scala:66)
>       at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>       at 
> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245)
>       at scala.collection.Iterator$class.foreach(Iterator.scala:742)
>       at scala.collection.AbstractIterator.foreach(Iterator.scala:1194)
>       at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
>       at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
>       at scala.collection.TraversableLike$class.map(TraversableLike.scala:245)
>       at scala.collection.AbstractTraversable.map(Traversable.scala:104)
>       at 
> kafka.integration.KafkaServerTestHarness$class.setUp(KafkaServerTestHarness.scala:66)
>       at 
> kafka.api.SslEndToEndAuthorizationTest.kafka$api$IntegrationTestHarness$$super$setUp(SslEndToEndAuthorizationTest.scala:24)
>       at 
> kafka.api.IntegrationTestHarness$class.setUp(IntegrationTestHarness.scala:58)
>       at 
> kafka.api.SslEndToEndAuthorizationTest.kafka$api$EndToEndAuthorizationTest$$super$setUp(SslEndToEndAuthorizationTest.scala:24)
>       at 
> kafka.api.EndToEndAuthorizationTest$class.setUp(EndToEndAuthorizationTest.scala:141)
>       at 
> kafka.api.SslEndToEndAuthorizationTest.setUp(SslEndToEndAuthorizationTest.scala:24)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:606)
>       at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
>       at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>       at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
>       at 
> org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:24)
>       at 
> org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
>       at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
>       at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
>       at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
>       at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
>       at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
>       at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
>       at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
>       at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
>       at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
>       at 
> org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecuter.runTestClass(JUnitTestClassExecuter.java:105)
>       at 
> org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecuter.execute(JUnitTestClassExecuter.java:56)
>       at 
> org.gradle.api.internal.tasks.testing.junit.JUnitTestClassProcessor.processTestClass(JUnitTestClassProcessor.java:64)
>       at 
> org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.processTestClass(SuiteTestClassProcessor.java:50)
>       at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:606)
>       at 
> org.gradle.messaging.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
>       at 
> org.gradle.messaging.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
>       at 
> org.gradle.messaging.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:32)
>       at 
> org.gradle.messaging.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:93)
>       at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
>       at 
> org.gradle.api.internal.tasks.testing.worker.TestWorker.processTestClass(TestWorker.java:106)
>       at sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:606)
>       at 
> org.gradle.messaging.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
>       at 
> org.gradle.messaging.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
>       at 
> org.gradle.messaging.remote.internal.hub.MessageHub$Handler.run(MessageHub.java:360)
>       at 
> org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:54)
>       at 
> org.gradle.internal.concurrent.StoppableExecutorImpl$1.run(StoppableExecutorImpl.java:40)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>       at java.lang.Thread.run(Thread.java:744)
> {code}
> Example of this failure:
> https://builds.apache.org/job/kafka-trunk-git-pr-jdk7/1555/testReport/junit/kafka.api/SslEndToEndAuthorizationTest/testNoGroupAcl/



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

Reply via email to