[
https://issues.apache.org/jira/browse/KAFKA-3134?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Yifan Ying updated KAFKA-3134:
------------------------------
Description:
I tried to initialize a KafkaConsumer object using with a null keyDeserializer
and a non-null valueDeserializer:
{code}
public KafkaConsumer(Properties properties, Deserializer<K> keyDeserializer,
Deserializer<V> valueDeserializer)
{code}
Then I got an exception as follows:
{code}
Caused by: org.apache.kafka.common.config.ConfigException: Missing required
configuration "value.deserializer" which has no default value.
at org.apache.kafka.common.config.ConfigDef.parse(ConfigDef.java:148)
at
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:49)
at
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:56)
at
org.apache.kafka.clients.consumer.ConsumerConfig.<init>(ConsumerConfig.java:336)
at
org.apache.kafka.clients.consumer.KafkaConsumer.<init>(KafkaConsumer.java:518)
.....
{code}
Then I went to ConsumerConfig.java file and found this block of code causing
the problem:
{code}
public static Map<String, Object> addDeserializerToConfig(Map<String, Object>
configs,
Deserializer<?>
keyDeserializer,
Deserializer<?>
valueDeserializer) {
Map<String, Object> newConfigs = new HashMap<String, Object>();
newConfigs.putAll(configs);
if (keyDeserializer != null)
newConfigs.put(KEY_DESERIALIZER_CLASS_CONFIG,
keyDeserializer.getClass());
if (keyDeserializer != null)
newConfigs.put(VALUE_DESERIALIZER_CLASS_CONFIG,
valueDeserializer.getClass());
return newConfigs;
}
public static Properties addDeserializerToConfig(Properties properties,
Deserializer<?>
keyDeserializer,
Deserializer<?>
valueDeserializer) {
Properties newProperties = new Properties();
newProperties.putAll(properties);
if (keyDeserializer != null)
newProperties.put(KEY_DESERIALIZER_CLASS_CONFIG,
keyDeserializer.getClass().getName());
if (keyDeserializer != null)
newProperties.put(VALUE_DESERIALIZER_CLASS_CONFIG,
valueDeserializer.getClass().getName());
return newProperties;
}
{code}
Instead of checking valueDeserializer, the code checks keyDeserializer every
time. So when keyDeserializer is null but valueDeserializer is not, the
valueDeserializer property will never get set.
was:
I tried to initialize a KafkaConsumer object using with a null keyDeserializer
and a non-null valueDeserializer:
{code}
public KafkaConsumer(Properties properties, Deserializer<K> keyDeserializer,
Deserializer<V> valueDeserializer)
{code}
Then I got an exception as follows:
{code}
Caused by: org.apache.kafka.common.config.ConfigException: Missing required
configuration "value.deserializer" which has no default value.
at org.apache.kafka.common.config.ConfigDef.parse(ConfigDef.java:148)
at
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:49)
at
org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:56)
at
org.apache.kafka.clients.consumer.ConsumerConfig.<init>(ConsumerConfig.java:336)
at
org.apache.kafka.clients.consumer.KafkaConsumer.<init>(KafkaConsumer.java:518)
.....
{code}
Then I went ConsumerConfig.java file and found this block of code causing the
problem:
{code}
public static Map<String, Object> addDeserializerToConfig(Map<String, Object>
configs,
Deserializer<?>
keyDeserializer,
Deserializer<?>
valueDeserializer) {
Map<String, Object> newConfigs = new HashMap<String, Object>();
newConfigs.putAll(configs);
if (keyDeserializer != null)
newConfigs.put(KEY_DESERIALIZER_CLASS_CONFIG,
keyDeserializer.getClass());
if (keyDeserializer != null)
newConfigs.put(VALUE_DESERIALIZER_CLASS_CONFIG,
valueDeserializer.getClass());
return newConfigs;
}
public static Properties addDeserializerToConfig(Properties properties,
Deserializer<?>
keyDeserializer,
Deserializer<?>
valueDeserializer) {
Properties newProperties = new Properties();
newProperties.putAll(properties);
if (keyDeserializer != null)
newProperties.put(KEY_DESERIALIZER_CLASS_CONFIG,
keyDeserializer.getClass().getName());
if (keyDeserializer != null)
newProperties.put(VALUE_DESERIALIZER_CLASS_CONFIG,
valueDeserializer.getClass().getName());
return newProperties;
}
{code}
Instead of checking valueDeserializer, the code checks keyDeserializer every
time. So when keyDeserializer is null but valueDeserializer is not, the
valueDeserializer property will never get set.
> Missing required configuration "value.deserializer" when initializing a
> KafkaConsumer with a valid "valueDeserializer"
> ----------------------------------------------------------------------------------------------------------------------
>
> Key: KAFKA-3134
> URL: https://issues.apache.org/jira/browse/KAFKA-3134
> Project: Kafka
> Issue Type: Bug
> Affects Versions: 0.9.0.0
> Reporter: Yifan Ying
>
> I tried to initialize a KafkaConsumer object using with a null
> keyDeserializer and a non-null valueDeserializer:
> {code}
> public KafkaConsumer(Properties properties, Deserializer<K> keyDeserializer,
> Deserializer<V> valueDeserializer)
> {code}
> Then I got an exception as follows:
> {code}
> Caused by: org.apache.kafka.common.config.ConfigException: Missing required
> configuration "value.deserializer" which has no default value.
> at org.apache.kafka.common.config.ConfigDef.parse(ConfigDef.java:148)
> at
> org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:49)
> at
> org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:56)
> at
> org.apache.kafka.clients.consumer.ConsumerConfig.<init>(ConsumerConfig.java:336)
> at
> org.apache.kafka.clients.consumer.KafkaConsumer.<init>(KafkaConsumer.java:518)
> .....
> {code}
> Then I went to ConsumerConfig.java file and found this block of code causing
> the problem:
> {code}
> public static Map<String, Object> addDeserializerToConfig(Map<String, Object>
> configs,
> Deserializer<?>
> keyDeserializer,
> Deserializer<?>
> valueDeserializer) {
> Map<String, Object> newConfigs = new HashMap<String, Object>();
> newConfigs.putAll(configs);
> if (keyDeserializer != null)
> newConfigs.put(KEY_DESERIALIZER_CLASS_CONFIG,
> keyDeserializer.getClass());
> if (keyDeserializer != null)
> newConfigs.put(VALUE_DESERIALIZER_CLASS_CONFIG,
> valueDeserializer.getClass());
> return newConfigs;
> }
> public static Properties addDeserializerToConfig(Properties properties,
> Deserializer<?>
> keyDeserializer,
> Deserializer<?>
> valueDeserializer) {
> Properties newProperties = new Properties();
> newProperties.putAll(properties);
> if (keyDeserializer != null)
> newProperties.put(KEY_DESERIALIZER_CLASS_CONFIG,
> keyDeserializer.getClass().getName());
> if (keyDeserializer != null)
> newProperties.put(VALUE_DESERIALIZER_CLASS_CONFIG,
> valueDeserializer.getClass().getName());
> return newProperties;
> }
> {code}
> Instead of checking valueDeserializer, the code checks keyDeserializer every
> time. So when keyDeserializer is null but valueDeserializer is not, the
> valueDeserializer property will never get set.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)