Hi Francois,

Thanks for the KIP! I sympathize with the issue you're facing and with
John's reluctance to let perfect be the enemy of good, and if KIP freeze
were tomorrow, I think this would be good enough. Given that we still have
some time to work with, I'd like to propose an alternative approach and see
what your thoughts are.

There are a few issues with the current client APIs that are closely
related to the KIP:
1. Too many constructors (there are currently four each for KafkaProducer
and KafkaConsumer, yet they all do basically the same thing)
2. Lack of type safety with interceptors (you have no way to enforce at
compile time that your ProducerInterceptor<String, Integer> is used with a
Serializer<String> and Serializer<Integer>, for example)
3. Inflexibility and inconsistency with instantiation of pluggable
interfaces (you can bring your own (de)serializers, but everything else
gets instantiated and configured for you at producer/consumer construction
time)

The KIP as it exists now will only address item 3, and will exacerbate item
1.

In addition, there are a few new issues introduced by the KIP as it exists
now:
1. Tighter coupling between the ProducerConfig/ConsumerConfig classes and
the KafkaProducer/KafkaConsumer classes. Any change we make in the future
that breaks either of these config classes in unexpected ways (but which
does not break the KafkaProducer/KafkaConsumer constructors that do not
accept these classes as parameters) will now have a much higher chance to
also break a user's entire producer/consumer application.
2. Complexity for users like yourself who would like to override behavior
in a ProducerConfig/ConsumerConfig in order to inject pre-instantiated
dependencies. The example in the KIP overrides
AbstractConfig::getConfiguredInstances [1] in order to achieve this. But
there are two other overloaded variants of getConfiguredInstances, and two
AbstractConfig::getConfiguredInstance methods that also exist. We'd either
need to establish a dependency graph between these methods (e.g., some
methods are guaranteed to invoke another overloaded variant) as part of the
public API for the AbstractConfig, or users would need to override every
single one of these methods in order to ensure that their code won't break
at runtime after bumping their Kafka version.

I think introducing the builder pattern for KafkaProducer and KafkaConsumer
would alleviate all of these issues. As a rough draft of what the API might
look like for KafkaProducer:

public class Builder<K, V> {
    private final Map<String, Object> props;
    private Serializer<K> keySerializer;
    private Serializer<V> valueSerializer;
    private List<ProducerInterceptor<K, V>> interceptors;
    private Map<String, Object> configuredInstances;
    private Map<String, List<Object>> configuredInstanceLists;

    public Builder(Map<String, Object> props) {
        this.props = props;
        this.interceptors = new ArrayList<>();
        this.configuredInstances = new HashMap<>();
        this.configuredInstanceLists = new HashMap<>();
    }

    // Use this serializer, if non-null
    // Will take precedence over any serializer class specified in the
properties for this producer
    public Builder withKeySerializer(Serializer<K> serializer) {
        this.keySerializer = serializer;
        return this;
    }

    public Builder withValueSerializer(Serializer<V> serializer) {
        this.valueSerializer = serializer;
        return this;
    }

    // Use these interceptors (has no effect if null)
    // Each must already be configured
    // Will be combined with any interceptor classes also specified in the
properties for this producer
    public Builder withInterceptors(List<ProducerInterceptor<K, V>>
interceptors) {
        if (interceptors != null) {
            this.interceptors.addAll(interceptors);
        }
        return this;
    }

    // Use this plugin instance, if non-null
    // Must already be configured
    // Will take precedence over any plugin class specified for the same
property in the properties for this producer (wording here needs work but
you get the idea)
    public Builder withConfiguredInstance(String property, Object instance)
{
        this.configuredInstances.put(property, instance);
        return this;
    }

    // Use these plugin instances (has no effect if null)
    // Each must already be configured
    // Will be combined with any plugin classes also specified for the same
property in the properties for this consumer
    public Builder withConfiguredInstances(String property, List<Object>
instances) {
        this.configuredInstanceLists.put(property, instances);
        return this;
    }

    public KafkaProducer<K, V> build() { ... }
}

Thoughts?

[1] -
https://kafka.apache.org/31/javadoc/org/apache/kafka/common/config/AbstractConfig.html#getConfiguredInstances(java.lang.String,java.lang.Class,java.util.Map)

Cheers,

Chris

On Mon, May 9, 2022 at 4:55 PM Bruno Cadonna <cado...@apache.org> wrote:

> Hi Francois,
>
> I think you can go ahead and call for votes.
>
> Could you please also clean up a little bit the KIP since it has still
> parts that refer to its first version? For example, "Compatibility,
> Deprecation, and Migration Plan" still mentions only two constructors.
> IMO you can also remove section "Public Interfaces" since it does not
> contain much information.
>
> Best,
> Bruno
>
> On 09.05.22 17:45, Bruno Cadonna wrote:
> > Hi Francois,
> >
> > You can open a PR and people can review it, but it must not be merged
> > until the KIP is approved.
> >
> > Best,
> > Bruno
> >
> > On 09.05.22 16:07, François Rosière wrote:
> >> Can a PR be dropped on Github or do we still need some approval first?
> >>
> >> Le dim. 8 mai 2022 à 06:08, John Roesler <vvcep...@apache.org> a écrit
> :
> >>
> >>> Thanks, François!
> >>>
> >>> Those changes look good to me.
> >>>
> >>> Thanks,
> >>> -John
> >>>
> >>> On Fri, May 6, 2022, at 13:51, François Rosière wrote:
> >>>> The KIP has been updated to reflect the last discussion
> >>>>
> >>>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882578#KIP832:Allowcreatingaproducer/consumerusingaproducer/consumerconfig-ProposedChanges
> >>>
> >>>>
> >>>>
> >>>> Le ven. 6 mai 2022 à 20:44, François Rosière
> >>>> <francois.rosi...@gmail.com>
> >>> a
> >>>> écrit :
> >>>>
> >>>>> Hello,
> >>>>>
> >>>>> No problem to also add a constructor taking the StreamsConfig in the
> >>>>> TopologyTestDriver.
> >>>>>
> >>>>> Summary about the changes to apply:
> >>>>>
> >>>>>     - Create 2 new constructors in KafkaProducer
> >>>>>     - Create a new constructor in KafkaConsumer and increase de
> >>> visibility
> >>>>>     of an existing one
> >>>>>     - Create a new constructor in TopologyTestDriver
> >>>>>
> >>>>> Kr,
> >>>>>
> >>>>> F.
> >>>>>
> >>>>> Le ven. 6 mai 2022 à 16:57, John Roesler <vvcep...@apache.org> a
> écrit
> >>> :
> >>>>>
> >>>>>> Thanks for the KIP, François!
> >>>>>>
> >>>>>> I'm generally in favor of your KIP, since you're
> >>>>>> proposing to follow the existing pattern of the
> >>>>>> constructors for both Producer and Consumer,
> >>>>>> but with the config object instead of Properties
> >>>>>> or Map configs. Also, because we already have
> >>>>>> this pattern in Streams, and we are just
> >>>>>> extending it to Producer and Consumer.
> >>>>>>
> >>>>>> Following on the KIP-378 discussion, I do still think
> >>>>>> this is somewhat of an abuse of the Config objects,
> >>>>>> and it would be better to have a formal dependency
> >>>>>> injection interface, but I also don't want to let perfect
> >>>>>> be the enemy of good. Since it looks like this approach
> >>>>>> works, and there is also some precedent for it already,
> >>>>>> I'd be inclined to approve it.
> >>>>>>
> >>>>>> Since KIP-378 didn't make it over the finish line, and it
> >>>>>> seems like a small expansion to your proposal, do you
> >>>>>> mind also adding the StreamsConfig to the
> >>>>>> TopologyTestDriver constructors? That way, we can go
> >>>>>> ahead and resolve both KIPs at once.
> >>>>>>
> >>>>>> Thank you,
> >>>>>> -John
> >>>>>>
> >>>>>>
> >>>>>> On Fri, May 6, 2022, at 06:06, François Rosière wrote:
> >>>>>>> To stay consistent with existing code, we should simply add 2
> >>>>>> constructors.
> >>>>>>> One with ser/deser and one without.
> >>>>>>> So that, users have the choice to use one or the other.
> >>>>>>> I updated the KIP accordingly.
> >>>>>>>
> >>>>>>> Le ven. 6 mai 2022 à 12:55, François Rosière <
> >>>>>> francois.rosi...@gmail.com> a
> >>>>>>> écrit :
> >>>>>>>
> >>>>>>>> On the other hand, the KafkaConsumer constructor with a config +
> >>>>>>>> serializer and deserializer already exists but is not public.
> >>>>>>>> It would also complexify a bit the caller to not have the
> >>>>>>>> serializer/deserializer exposed at constructor level.
> >>>>>>>>
> >>>>>>>> Once the KIP would have been implemented, for streams, instead of
> >>>>>> having a
> >>>>>>>> custom config (already possible), I may simply define a custom
> >>>>>>>> KafkaClientSupplier reusing the custom configs of both the
> producer
> >>>>>> and the
> >>>>>>>> consumer.
> >>>>>>>> This supplier currently creates producers and consumers using the
> >>>>>>>> constructors with a map of config + serializer/deserializer.
> >>>>>>>>
> >>>>>>>> So, it seems it's easier to have the constructor with 3
> parameters.
> >>>>>> But in
> >>>>>>>> any case, it will work if the config can be accessed...
> >>>>>>>>
> >>>>>>>> Le ven. 6 mai 2022 à 12:14, François Rosière <
> >>>>>> francois.rosi...@gmail.com>
> >>>>>>>> a écrit :
> >>>>>>>>
> >>>>>>>>> Hello,
> >>>>>>>>>
> >>>>>>>>> We may create a constructor with a single parameter which is the
> >>>>>> config
> >>>>>>>>> but then, I would need to give the serializer/deserializer by
> also
> >>>>>>>>> overriding the config.
> >>>>>>>>> Like I would do for the interceptors.
> >>>>>>>>> So, no real opinion on that, both solutions are ok for me.
> >>>>>>>>> Maybe easier to take the approach of the single parameter.
> >>>>>>>>>
> >>>>>>>>> Hope it respond to the question.
> >>>>>>>>>
> >>>>>>>>> Kr,
> >>>>>>>>>
> >>>>>>>>> F.
> >>>>>>>>>
> >>>>>>>>> Le ven. 6 mai 2022 à 11:59, Bruno Cadonna <cado...@apache.org> a
> >>>>>> écrit :
> >>>>>>>>>
> >>>>>>>>>> Hi Francois,
> >>>>>>>>>>
> >>>>>>>>>> Thank you for updating the KIP!
> >>>>>>>>>>
> >>>>>>>>>> Now the motivation of the KIP is much clearer.
> >>>>>>>>>>
> >>>>>>>>>> I would still be interested in:
> >>>>>>>>>>
> >>>>>>>>>>   >> 2. Why do you only want to change/add the constructors that
> >>> take
> >>>>>> the
> >>>>>>>>>>   >> properties objects and de/serializers and you do not also
> >>> want to
> >>>>>>>>>>   >> add/change the constructors that take only the properties?
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Best,
> >>>>>>>>>> Bruno
> >>>>>>>>>>
> >>>>>>>>>> On 05.05.22 23:15, François Rosière wrote:
> >>>>>>>>>>> Hello Bruno,
> >>>>>>>>>>>
> >>>>>>>>>>> The KIP as been updated. Feel free to give more feedbacks and I
> >>>>>> will
> >>>>>>>>>>> complete accordingly.
> >>>>>>>>>>>
> >>>>>>>>>>> Kr,
> >>>>>>>>>>>
> >>>>>>>>>>> F.
> >>>>>>>>>>>
> >>>>>>>>>>> Le jeu. 5 mai 2022 à 22:22, Bruno Cadonna <cado...@apache.org>
> >>> a
> >>>>>>>>>> écrit :
> >>>>>>>>>>>
> >>>>>>>>>>>> Hi Francois,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks for the KIP!
> >>>>>>>>>>>>
> >>>>>>>>>>>> Here my first feedback:
> >>>>>>>>>>>>
> >>>>>>>>>>>> 1. Could you please extend the motivation section, so that it
> >>> is
> >>>>>> clear
> >>>>>>>>>>>> for a non-Spring dev why the change is needed? Usually, a
> >>>>>> motivation
> >>>>>>>>>>>> section benefits a lot from an actual example.
> >>>>>>>>>>>> Extending the motivation section would also make the KIP more
> >>>>>>>>>>>> self-contained which is important IMO since this is kind of a
> >>> log
> >>>>>> of
> >>>>>>>>>> the
> >>>>>>>>>>>> major changes to Kafka. Descriptions of major changes should
> >>> not
> >>>>>>>>>>>> completely depend on external links (which may become dead in
> >>>>>> future).
> >>>>>>>>>>>> Referencing external resources to point to more details or
> give
> >>>>>>>>>> context
> >>>>>>>>>>>> is useful, though.
> >>>>>>>>>>>>
> >>>>>>>>>>>> 2. Why do you only want to change/add the constructors that
> >>> take
> >>>>>> the
> >>>>>>>>>>>> properties objects and de/serializers and you do not also want
> >>> to
> >>>>>>>>>>>> add/change the constructors that take only the properties?
> >>>>>>>>>>>>
> >>>>>>>>>>>> 3. I found the following stalled KIP whose motivation is
> really
> >>>>>>>>>> similar
> >>>>>>>>>>>> to yours:
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>
> >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-378%3A+Enable+Dependency+Injection+for+Kafka+Streams+handlers
> >>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> That KIP is also the reason why Kafka Streams still has the
> >>>>>>>>>> constructors
> >>>>>>>>>>>> with the StreamsConfig parameter. Maybe you want to mention
> >>> this
> >>>>>> KIP
> >>>>>>>>>> in
> >>>>>>>>>>>> yours or even incorporate the remaining topology test driver
> >>> API
> >>>>>>>>>> changes
> >>>>>>>>>>>> in your KIP.
> >>>>>>>>>>>> Some related links:
> >>>>>>>>>>>> -
> >>>>>> https://github.com/apache/kafka/pull/5344#issuecomment-413350338
> >>>>>>>>>>>> - https://github.com/apache/kafka/pull/10484
> >>>>>>>>>>>> - https://issues.apache.org/jira/browse/KAFKA-6386
> >>>>>>>>>>>>
> >>>>>>>>>>>> Best,
> >>>>>>>>>>>> Bruno
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 04.05.22 22:26, François Rosière wrote:
> >>>>>>>>>>>>> Hi all,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> KIP-832 has been created to allow implementing Spring managed
> >>>>>>>>>>>> interceptors
> >>>>>>>>>>>>> for Producers and Consumers.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> At the moment, interceptors are given as configuration
> classes
> >>>>>> to the
> >>>>>>>>>>>>> producer and consumer configurations. So, the idea here would
> >>> be
> >>>>>> to
> >>>>>>>>>>>> create
> >>>>>>>>>>>>> 2 new constructors to allow using a Producer and Consumer
> >>>>>>>>>> configuration
> >>>>>>>>>>>>> instead of properties or a key value map of configurations
> >>>>>> entries.
> >>>>>>>>>>>>> Interceptors could then be given as instances by overriding a
> >>>>>> config
> >>>>>>>>>>>> method.
> >>>>>>>>>>>>> More details can be found in the Spring issue.
> >>>>>>>>>>>>> https://github.com/spring-projects/spring-kafka/issues/2244
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Any feedback, proposal, vote for this KIP would be more than
> >>>>>> welcome.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Kind regards,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Francois R.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Le lun. 2 mai 2022 à 21:05, François Rosière <
> >>>>>>>>>> francois.rosi...@gmail.com>
> >>>>>>>>>>>> a
> >>>>>>>>>>>>> écrit :
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Kip link:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>
> >>>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882578
> >>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>
> >>>>>
> >>>
> >>
>

Reply via email to