You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Margaret Figura (JIRA)" <ji...@apache.org> on 2019/03/06 17:30:00 UTC

[jira] [Created] (KAFKA-8054) KafkaProducer(Properties, Serializer, Serializer) fails when serializers are lambdas

Margaret Figura created KAFKA-8054:
--------------------------------------

             Summary: KafkaProducer(Properties, Serializer<K>, Serializer<V>) fails when serializers are lambdas
                 Key: KAFKA-8054
                 URL: https://issues.apache.org/jira/browse/KAFKA-8054
             Project: Kafka
          Issue Type: Bug
          Components: producer 
    Affects Versions: 1.1.0
            Reporter: Margaret Figura


I refactored some code that creates a KafkaProducer. Previously, it set configuration parameters in a HashMap, but I changed it to use a Properties instead. This calls a different constructor, and that constructor fails in some cases.

In this example, we create a KafkaProducer with the Map constructor which succeeds, and the Properties constructor which fails:
{code:java}
package test;

import java.util.HashMap;
import java.util.Map;
import java.util.Properties;

import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.serialization.Serializer;

public class KafkaProducerProperties
{
    @FunctionalInterface
    public interface KafkaSerializer<T> extends Serializer<T>
    {        
        @Override
        default public void configure(Map<String, ?> configs, boolean isKey){};
        @Override
        default public void close() {};
    }
    
    private final static KafkaSerializer<String> stringSerializer = (topic, key) -> {
        return null; // TODO: do something :)
    };
    
    public static void main(String[] args)
    {
        HashMap<String, Object> hashConfig = new HashMap<>();
        Properties propConfig = new Properties();
        
        hashConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "1.2.3.4:1234");
        propConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "1.2.3.4:1234");
        
        new KafkaProducer<>(hashConfig, stringSerializer, stringSerializer).close(); // Works fine
        new KafkaProducer<>(propConfig, stringSerializer, stringSerializer).close(); // Throws ConfigException
    }
}
{code}
 

The exception thrown from the Properties-based constructor is:
{noformat}
Exception in thread "main" org.apache.kafka.common.config.ConfigException: Invalid value test.KafkaProducerProperties$$Lambda$1/1296064247 for configuration key.serializer: Class test.KafkaProducerProperties$$Lambda$1/1296064247 could not be found.
	at org.apache.kafka.common.config.ConfigDef.parseType(ConfigDef.java:724)
	at org.apache.kafka.common.config.ConfigDef.parseValue(ConfigDef.java:469)
	at org.apache.kafka.common.config.ConfigDef.parse(ConfigDef.java:462)
	at org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:62)
	at org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:75)
	at org.apache.kafka.clients.producer.ProducerConfig.<init>(ProducerConfig.java:365)
	at org.apache.kafka.clients.producer.KafkaProducer.<init>(KafkaProducer.java:318)
	at test.KafkaProducerProperties.main(KafkaProducerProperties.java:36)
{noformat}
Note that the Properties constructor seems to work fine if I create a Serializer instance in a 'normal way', but if it's implemented via a functional interface it throws this error.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)