Hello Wladimir,

Thanks for the replies.

What do you mean by "the community has opted for the second more complex
solution"? Could you elaborate a bit more?

Guozhang


On Sun, Mar 17, 2019 at 3:45 PM Wladimir Schmidt <wlsc....@gmail.com> wrote:

> Hi Matthias,
>
> Sorry, due to other commitments I haven't started the other
> implementation yet.
> In the meantime, the community has opted for the second, more complex
> solution.
> I already had ideas in this regard, but their elaboration needs to be
> discussed more.
>
>
> Best greetings,
> Wladimir
>
>
> On 21-Feb-19 09:33, Matthias J. Sax wrote:
> > Hi Wladimir,
> >
> > what is the status of this KIP?
> >
> > -Matthias
> >
> > On 1/9/19 4:17 PM, Guozhang Wang wrote:
> >> Hello Wladimir,
> >>
> >> Just checking if you are still working on this KIP. We have the 2.2 KIP
> >> freeze deadline by 24th this month, and it'll be great to complete this
> KIP
> >> by then so 2.2.0 release could have this feature.
> >>
> >>
> >> Guozhang
> >>
> >> On Mon, Dec 3, 2018 at 11:26 PM Guozhang Wang <wangg...@gmail.com>
> wrote:
> >>
> >>> Hello Wladimir,
> >>>
> >>> I've thought about the two options and I think I'm sold on the second
> >>> option and actually I think it is better generalize it to be
> potentially
> >>> used for other clients (producer, consumer) as while since they also
> have
> >>> similar dependency injection requests for metrics reporter,
> partitioner,
> >>> partition assignor etc.
> >>>
> >>> So I'd suggest we add the following to AbstractConfig directly (note I
> >>> intentionally renamed the class to ConfiguredInstanceFactory to be
> used for
> >>> other clients as well):
> >>>
> >>> ```
> >>> AbstractConfig(ConfigDef definition, Map<?, ?> originals,
> >>> ConfiguredInstanceFactory, boolean doLog)
> >>> ```
> >>>
> >>> And then in StreamsConfig add:
> >>>
> >>> ```
> >>> StreamsConfig(Map<?, ?> props, ConfiguredInstanceFactory)
> >>> ```
> >>>
> >>> which would call the above AbstractConfig constructor (we can leave to
> >>> core team to decide when they want to add for producer and consumer);
> >>>
> >>> And in KafkaStreams / TopologyTestDriver we can add one overloaded
> >>> constructor each that includes all the parameters including the
> >>> ConfiguredInstanceFactory --- for those who only want `factory` but not
> >>> `client-suppliers` for example, they can set it to `null` and the
> streams
> >>> library will just use the default one.
> >>>
> >>>
> >>> Guozhang
> >>>
> >>>
> >>> On Sun, Dec 2, 2018 at 12:13 PM Wladimir Schmidt <wlsc....@gmail.com>
> >>> wrote:
> >>>
> >>>> Hello Guozhang,
> >>>> sure, the first approach is very straight-forward and allows minimal
> >>>> changes to the Kafka Streams API.
> >>>> On the other hand, second approach with the interface implementation
> >>>> looks more cleaner to me.
> >>>> I totally agree that this should be first discussed before will be
> >>>> implemented.
> >>>>
> >>>> Thanks, Wladimir
> >>>>
> >>>>
> >>>> On 17-Nov-18 23:37, Guozhang Wang wrote:
> >>>>
> >>>> Hello folks,
> >>>>
> >>>> I'd like to revive this thread for discussion. After reading the
> previous
> >>>> emails I think I'm still a bit leaning towards re-enabling to pass in
> >>>> StreamsConfig to Kafka Streams constructors compared with a
> >>>> ConfiguredStreamsFactory as additional parameters to overloaded
> >>>> KafkaStreams constructors: although the former seems less cleaner as
> it
> >>>> requires users to read through the usage of AbstractConfig to know
> how to
> >>>> use it in their frameworks, this to me is a solvable problem through
> >>>> documentations, plus AbstractConfig is a public interface already and
> hence
> >>>> the additional ConfiguredStreamsFactory to me is really a bit
> overlapping
> >>>> in functionality.
> >>>>
> >>>>
> >>>> Guozhang
> >>>>
> >>>>
> >>>>
> >>>> On Sun, Oct 21, 2018 at 1:41 PM Wladimir Schmidt <wlsc....@gmail.com>
> <wlsc....@gmail.com> wrote:
> >>>>
> >>>>
> >>>> Hi Damian,
> >>>>
> >>>> The first approach was added only because it had been initially
> proposed
> >>>> in my pull request,
> >>>> which started a discussion and thus, the KIP-378 was born.
> >>>>
> >>>> Yes, I would like to have something "injectable". In this regard, a
> >>>> `ConfiguredStreamsFactory` (name is a subject to discussion)
> >>>> is a good option to be introduced into `KafkaStreams` constructor.
> >>>>
> >>>> Even though, I consider the second approach to be cleaner, it
> involves a
> >>>> certain amount of refactoring of the streams library.
> >>>> The first approach, on the contrary, adds (or removes deprecated
> >>>> annotation, if the method has not been removed yet) only additional
> >>>> constructors with
> >>>> considerably less intervention into a streams library (no changes,
> which
> >>>> would break an API. Please see a pull request:
> https://github.com/apache/kafka/pull/5344).
> >>>>
> >>>> Thanks
> >>>> Wladimir
> >>>>
> >>>> On 10-Oct-18 15:51, Damian Guy wrote:
> >>>>
> >>>> Hi Wladimir,
> >>>>
> >>>> Of the two approaches in the KIP - i feel the second approach is
> cleaner.
> >>>> However, am i correct in assuming that you want to have the
> >>>> `ConfiguredStreamsFactory` as a ctor arg in `StreamsConfig` so that
> >>>>
> >>>> Spring
> >>>>
> >>>> can inject this for you?
> >>>>
> >>>> Otherwise you could just put the ApplicationContext as a property in
> the
> >>>> config and then use that via the configure method of the appropriate
> >>>> handler to get your actual handler.
> >>>>
> >>>> Thanks,
> >>>> Damian
> >>>>
> >>>> On Tue, 9 Oct 2018 at 01:55, Guozhang Wang <wangg...@gmail.com> <
> wangg...@gmail.com> wrote:
> >>>>
> >>>>
> >>>> John, thanks for the explanation, now it makes much more sense to me.
> >>>>
> >>>> As for the concrete approach, to me it seems the first option requires
> >>>>
> >>>> less
> >>>>
> >>>> changes than the second (ConfiguredStreamsFactory based) approach,
> >>>>
> >>>> whereas
> >>>>
> >>>> the second one requires an additional interface that is overlapping
> with
> >>>> the AbstractConfig.
> >>>>
> >>>> I'm aware that in KafkaProducer / KafkaConsumer we do not have public
> >>>> constructors for taking a ProducerConfig or ConsumerConfig directly,
> and
> >>>> anyone using Spring can share how you've worked around it by far? If
> it
> >>>>
> >>>> is
> >>>>
> >>>> very awkward I'm not against just adding the XXXConfigs to the
> >>>>
> >>>> constructors
> >>>>
> >>>> directly.
> >>>>
> >>>> Guozhang
> >>>>
> >>>> On Fri, Oct 5, 2018 at 1:48 PM, John Roesler <j...@confluent.io> <
> j...@confluent.io> wrote:
> >>>>
> >>>>
> >>>> Hi Wladimir,
> >>>>
> >>>> Thanks for the KIP!
> >>>>
> >>>> As I mentioned in the PR discussion, I personally prefer not to
> >>>>
> >>>> recommend
> >>>>
> >>>> overriding StreamsConfig for this purpose.
> >>>>
> >>>> It seems like a person wishing to create a DI shim would have to
> >>>>
> >>>> acquire
> >>>>
> >>>> quite a deep understanding of the class and its usage to figure out
> >>>>
> >>>> what
> >>>>
> >>>> exactly to override to accomplish their goals without breaking
> >>>>
> >>>> everything.
> >>>>
> >>>> I'm honestly impressed with the method you came up with to create your
> >>>> Spring/Streams shim.
> >>>>
> >>>> I think we can make to path for the next person smoother by going with
> >>>> something more akin to the ConfiguredStreamsFactory. This is a
> >>>>
> >>>> constrained
> >>>>
> >>>> interface that tells you exactly what you have to implement to create
> >>>>
> >>>> such
> >>>>
> >>>> a shim.
> >>>>
> >>>> A few thoughts:
> >>>> 1. it seems like we can keep all the deprecated constructors still
> >>>> deprecated
> >>>>
> >>>> 2. we could add just one additional constructor to each of
> KafkaStreams
> >>>>
> >>>> and
> >>>>
> >>>> TopologyTestDriver to still take a Properties, but also your new
> >>>> ConfiguredStreamsFactory
> >>>>
> >>>> 3. I don't know if I'm sold on the name ConfiguredStreamsFactory,
> since
> >>>>
> >>>> it
> >>>>
> >>>> does not produce configured streams. Instead, it produces configured
> >>>> instances... How about ConfiguredInstanceFactory?
> >>>>
> >>>> 4. if I understand the usage correctly, it's actually a pretty small
> >>>>
> >>>> number
> >>>>
> >>>> of classes that we actually make via getConfiguredInstance. Offhand, I
> >>>>
> >>>> can
> >>>>
> >>>> think of the key/value Serdes, the deserialization exception handler,
> >>>>
> >>>> and
> >>>>
> >>>> the production exception handler.
> >>>> Perhaps, instead of maintaining a generic "class instantiator", we
> >>>>
> >>>> could
> >>>>
> >>>> explore a factory interface that just has methods for creating exactly
> >>>>
> >>>> the
> >>>>
> >>>> kinds of things we need to create. In fact, we already have something
> >>>>
> >>>> like
> >>>>
> >>>> this: org.apache.kafka.streams.KafkaClientSupplier . Do you think we
> >>>>
> >>>> could
> >>>>
> >>>> just add some more methods to that interface (and maybe rename it)
> >>>>
> >>>> instead?
> >>>>
> >>>> Thanks,
> >>>> -John
> >>>>
> >>>> On Fri, Oct 5, 2018 at 3:31 PM John Roesler <j...@confluent.io> <
> j...@confluent.io> wrote:
> >>>>
> >>>>
> >>>> Hi Guozhang,
> >>>>
> >>>> I'm going to drop in a little extra context from the preliminary PR
> >>>> discussion (https://github.com/apache/kafka/pull/5344).
> >>>>
> >>>> The issue isn't that it's impossible to use Streams within a Spring
> >>>>
> >>>> app,
> >>>>
> >>>> just that the interplay between our style of
> >>>>
> >>>> construction/configuration
> >>>>
> >>>> and
> >>>>
> >>>> Spring's is somewhat awkward compared to the normal experience with
> >>>> dependency injection.
> >>>>
> >>>> I'm guessing users of dependency injection would not like the approach
> >>>>
> >>>> you
> >>>>
> >>>> offered. I believe it's commonly considered an antipattern when using
> >>>>
> >>>> DI
> >>>>
> >>>> frameworks to pass the injector directly into the class being
> >>>>
> >>>> constructed.
> >>>>
> >>>> Wladimir has also offered an alternative usage within the current
> >>>>
> >>>> framework
> >>>>
> >>>> of injecting pre-constructed dependencies into the Properties, and
> >>>>
> >>>> then
> >>>>
> >>>> retrieving and casting them inside the configured class.
> >>>>
> >>>> It seems like this KIP is more about offering a more elegant interface
> >>>>
> >>>> to
> >>>>
> >>>> DI users.
> >>>>
> >>>> One of the points that Wladimir raised on his PR discussion was the
> >>>>
> >>>> desire
> >>>>
> >>>> to configure the classes in a typesafe way in the constructor (thus
> >>>> allowing the use of immutable classes).
> >>>>
> >>>> With this KIP, it would be possible for a DI user to:
> >>>> 1. register a Streams-Spring or Streams-Guice (etc) "plugin" (via
> >>>>
> >>>> either
> >>>>
> >>>> of the mechanisms he proposed)
> >>>> 2. simply make the Serdes, exception handlers, etc, available on the
> >>>>
> >>>> class
> >>>>
> >>>> path with the DI annotations
> >>>> 3. start the app
> >>>>
> >>>> There's no need to mess with passing dependencies (or the injector)
> >>>> through the properties.
> >>>>
> >>>> Sorry for "injecting" myself into your discussion, but it took me a
> >>>>
> >>>> while
> >>>>
> >>>> in the PR discussion to get to the bottom of the issue, and I wanted
> >>>>
> >>>> to
> >>>>
> >>>> spare you the same.
> >>>>
> >>>> I'll respond separately with my feedback on the KIP.
> >>>>
> >>>> Thanks,
> >>>> -John
> >>>>
> >>>> On Sun, Sep 30, 2018 at 2:31 PM Guozhang Wang <wangg...@gmail.com> <
> wangg...@gmail.com>
> >>>>
> >>>> wrote:
> >>>>
> >>>> Hello Wladimir,
> >>>>
> >>>> Thanks for proposing the KIP. I think the injection can currently be
> >>>>
> >>>> done
> >>>>
> >>>> by passing in the key/value pair directly into the properties which
> >>>>
> >>>> can
> >>>>
> >>>> then be accessed from the `ProcessorContext#appConfigs` or
> >>>> `#appConfigsWithPrefix`. For example, when constructing the
> >>>>
> >>>> properties
> >>>>
> >>>> you
> >>>>
> >>>> can:
> >>>>
> >>>> ```
> >>>> props.put(myProp1, myValue1);
> >>>> props.put(myProp2, myValue1);
> >>>> props.put("my_app_context", appContext);
> >>>>
> >>>> KafkaStreams myApp = new KafkaStreams(topology, props);
> >>>>
> >>>> // and then in your processor, on the processor where you want to
> >>>> construct
> >>>> the injected handler:
> >>>>
> >>>> Map<String, Object> appProps = processorContext.appConfigs();
> >>>> ApplicationContext appContext = appProps.get("my_app_context");
> >>>> MyHandler myHandler =
> >>>> applicationContext.getBeanNamesForType(MyHandlerClassType);
> >>>> ```
> >>>>
> >>>> Does that work for you?
> >>>>
> >>>> Guozhang
> >>>>
> >>>>
> >>>> On Sun, Sep 30, 2018 at 6:56 AM, Dongjin Lee <dong...@apache.org> <
> dong...@apache.org>
> >>>>
> >>>> wrote:
> >>>>
> >>>> Hi Wladimir,
> >>>>
> >>>> Thanks for your great KIP. Let me have a look. And let's discuss
> >>>>
> >>>> this
> >>>>
> >>>> KIP
> >>>>
> >>>> in depth after the release of 2.1.0. (The committers are very busy
> >>>>
> >>>> for
> >>>>
> >>>> it.)
> >>>>
> >>>> Best,
> >>>> Dongjin
> >>>>
> >>>> On Sun, Sep 30, 2018 at 10:49 PM Wladimir Schmidt <
> >>>>
> >>>> wlsc....@gmail.com
> >>>>
> >>>> wrote:
> >>>>
> >>>>
> >>>> Dear colleagues,
> >>>>
> >>>> I am happy to inform you that I have just finished my first KIP
> >>>> (KIP-378: Enable Dependency Injection for Kafka Streams handlers
> >>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>
> >>>> 378%3A+Enable+Dependency+Injection+for+Kafka+Streams+handlers
> >>>>
> >>>> ).
> >>>>
> >>>> Your feedback on this submission would be highly appreciated.
> >>>>
> >>>> Best Regards,
> >>>> Wladimir Schmidt
> >>>>
> >>>>
> >>>> --
> >>>> *Dongjin Lee*
> >>>>
> >>>> *A hitchhiker in the mathematical world.*
> >>>>
> >>>> *github:  <http://goog_969573159/> <http://goog_969573159/>
> github.com/dongjinleekr<http://github.com/dongjinleekr> <
> http://github.com/dongjinleekr>linkedin:
> >>>>
> >>>> kr.linkedin.com/in/dongjinleekr
> >>>>
> >>>> <http://kr.linkedin.com/in/dongjinleekr> <
> http://kr.linkedin.com/in/dongjinleekr>slideshare:
> www.slideshare.net/dongjinleekr<http://www.slideshare.net/dongjinleekr> <
> http://www.slideshare.net/dongjinleekr>*
> >>>>
> >>>>
> >>>> --
> >>>> -- Guozhang
> >>>>
> >>>>
> >>>> --
> >>>> -- Guozhang
> >>>>
> >>>>
> >>>>
> >>> --
> >>> -- Guozhang
> >>>
> >>
>


-- 
-- Guozhang

Reply via email to