Github user srdo commented on a diff in the pull request:
https://github.com/apache/storm/pull/2428#discussion_r151916643
--- Diff:
external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java
---
@@ -292,17 +292,21 @@ private Builder(String bootstrapServers,
SerializableDeserializer<K> keyDes, Cla
this.subscription = subscription;
this.translator = new DefaultRecordTranslator<>();
- if (keyDesClazz != null) {
-
this.kafkaProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDesClazz);
- }
- if (keyDes != null) {
-
this.kafkaProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
keyDes.getClass());
- }
- if (valueDesClazz != null) {
-
this.kafkaProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
valueDesClazz);
+ if
(!this.kafkaProps.containsKey(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)) {
--- End diff --
Yes, I'll explain. In https://github.com/apache/storm/pull/2155 I changed
the KafkaSpoutConfig API a bunch to try to avoid having custom methods for
properties that users should just set via kafkaProps. Part of the change
removes keyDes and keyDesClass, and tells users to set the corresponding
properties in kafkaProps instead.
When the changes were backported, I deprecated all constructors or methods
referring to those fields, and switched the KafkaSpoutConfig.builder
convenience methods to use a constructor that just sets the right properties in
kafkaProps. It turns out this is a breaking change for users that build a
KafkaSpoutConfig and use `getKeyDeserializer` or `getValueDeserializer` for
anything, because they are now null when using the convenience builders, where
they defaulted to StringDeserializers before.
In order to retain backwards compatibility, the builders have to set the
key/value deserializer fields to StringDeserializer again. I still want to get
rid of the fields though, so to allow users to switch to using kafkaProps
instead, we'll only use the fields if the corresponding properties in
kafkaProps are not set. If we set the properties based on the fields
unconditionally, we would overwrite the deserializer settings for users that
set the properties in kafkaProps.
> What happens if this if statement is false
If the expression is false, the field settings are ignored. This is the
behavior I think we want, since it means the user must have set the right
property in kafkaProps. The consequence of this is a mismatch between what is
in kafkaProps, and what is set in the key deserializer field. I suppose we
could overwrite the key deserializer field with whatever is in kafkaProps to
resolve the conflict?
>The two ifs bellow, on lines 296 and 299, I think they can possibly be
both true and with different values, if you are dealing with subtypes. If so,
what happens in that case?
We would use the deserializer from the keyDes field. I think this kind of
ambiguity is pretty nasty, but it's consistent with previous behavior.
---