Joe O'Connor created KAFKA-3218:
-----------------------------------

             Summary: Kafka-0.9.0.0 does not work as OSGi module
                 Key: KAFKA-3218
                 URL: https://issues.apache.org/jira/browse/KAFKA-3218
             Project: Kafka
          Issue Type: Bug
          Components: clients
    Affects Versions: 0.9.0.0
         Environment: Apache Felix OSGi container
jdk_1.8.0_60
            Reporter: Joe O'Connor


KAFKA-2295 changed all Class.forName() calls to use 
currentThread().getContextClassLoader() instead of the default "classloader 
that loaded the current class". 

OSGi loads each module's classes using a separate classloader so this is now 
broken.

Steps to reproduce: 
# install the kafka-clients servicemix OSGi module 0.9.0.0_1
# attempt to initialize the Kafka producer client from Java code 

Expected results: 
- call to "new KafkaProducer()" succeeds

Actual results: 
- "new KafkaProducer()" throws ConfigException:
{quote}        Suppressed: java.lang.Exception: Error starting bundle54: 
Activator start error in bundle com.openet.testcase.ContextClassLoaderBug [54].
                at 
org.apache.karaf.bundle.command.BundlesCommand.doExecute(BundlesCommand.java:66)
                ... 12 more
        Caused by: org.osgi.framework.BundleException: Activator start error in 
bundle com.openet.testcase.ContextClassLoaderBug [54].
                at 
org.apache.felix.framework.Felix.activateBundle(Felix.java:2276)
                at org.apache.felix.framework.Felix.startBundle(Felix.java:2144)
                at 
org.apache.felix.framework.BundleImpl.start(BundleImpl.java:998)
                at 
org.apache.karaf.bundle.command.Start.executeOnBundle(Start.java:38)
                at 
org.apache.karaf.bundle.command.BundlesCommand.doExecute(BundlesCommand.java:64)
                ... 12 more
        Caused by: java.lang.ExceptionInInitializerError
                at 
org.apache.kafka.clients.producer.KafkaProducer.<init>(KafkaProducer.java:156)
                at com.openet.testcase.Activator.start(Activator.java:16)
                at 
org.apache.felix.framework.util.SecureAction.startActivator(SecureAction.java:697)
                at 
org.apache.felix.framework.Felix.activateBundle(Felix.java:2226)
                ... 16 more
        *Caused by: org.apache.kafka.common.config.ConfigException: Invalid 
value org.apache.kafka.clients.producer.internals.DefaultPartitioner for 
configuration partitioner.class: Class* 
*org.apache.kafka.clients.producer.internals.DefaultPartitioner could not be 
found.*
                at 
org.apache.kafka.common.config.ConfigDef.parseType(ConfigDef.java:255)
                at 
org.apache.kafka.common.config.ConfigDef.define(ConfigDef.java:78)
                at 
org.apache.kafka.common.config.ConfigDef.define(ConfigDef.java:94)
                at 
org.apache.kafka.clients.producer.ProducerConfig.<clinit>(ProducerConfig.java:206)
{quote}


Workaround is to call "currentThread().setContextClassLoader(null)" before 
initializing the kafka producer.

Possible fix is to catch ClassNotFoundException at ConfigDef.java:247 and retry 
the Class.forName() call with the default classloader. However with this fix 
there is still a problem at AbstractConfig.java:206,  where the newInstance() 
call succeeds but "instanceof" is false because the classes were loaded by 
different classloaders.

Testcase attached, see README.txt for instructions.

See also SM-2743



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

Reply via email to