Thanks Guozhang.

This makes sense. I am still wondering about wrapped serdes:

> and if it is a wrapper serde, also print its inner
>>> serde name

How can our default implementation of `TopologyDescriber` know if it's a
wrapped serde or not? Furthermore, how do wrapped serdes expose their
inner serdes?

I am also not sure what the purpose of TopologyDescriber is? Would it
mabye be better to add new interface `Serdes` can implement instead?


-Matthias



On 5/18/20 9:24 PM, Guozhang Wang wrote:
> Bruno, Matthias:
> 
> Thanks for your inputs. After some thoughts I've decide to update my
> proposal in the following way:
> 
> 1. Store#serdes() would return a "Map<String, String>"
> 
> 2. Topology's description would be independent of whether it is generated
> from `StreamsBuilder#build(props)` or `StreamsBuilder#build()`, and if the
> serde is not known we would use "<unknown>" as the default value.
> 
> 3. Add `List<String> TopologyDescription#sourceTopics() / sinkTopics() /
> repartitionTopics() / changelogTopics()` and for pattern / topic-extractor
> we would use fixed format of "<pattern:regex>" and
> "<dynamic:extractor-class-name>".
> 
> 
> I will try to implement this in my existing PR and after I've confirmed it
> works, I will update the final wiki for voting.
> 
> 
> Guozhang
> 
> 
> On Mon, May 18, 2020 at 9:13 PM Guozhang Wang <wangg...@gmail.com> wrote:
> 
>> Hello Andy,
>>
>> Thanks a lot for your comments! I do not mind at all :)
>>
>> I think that's a valid point, what I have in mind is to expose an
>> interface which can be optionally overridden in the overridden describe()
>> call:
>>
>> Topology#describe(final TopologyDescriber)
>>
>> Interface TopologyDescriber {
>>
>>     default describeSerde(final Serde);
>>
>>     default describeSerializer(final Serializer);
>>
>>     default describeDeserializer(final Serializer);
>> }
>>
>> And we would expose a DefaultTopologyDescriber class that just print the
>> serde class names -- and if it is a wrapper serde, also print its inner
>> serde name.
>>
>> Guozhang
>>
>>
>> On Mon, May 11, 2020 at 12:13 PM Andy Coates <a...@confluent.io> wrote:
>>
>>> Hi Guozhang,
>>>
>>> Thanks for writing this up. I’m very interested to see this, so I hope
>>> you don’t mind me commenting.
>>>
>>> I’ve only really one comment to make, and that’s on the text printed for
>>> the serde classes:
>>>
>>> As I understand it, the name will either come from the passed in config,
>>> or may default to “unknown”, or may be obtained from the instances passed
>>> while building the topology. It’s this latter case that interests me.
>>> Where you have an actual serde instance could we not output more
>>> information?
>>>
>>> The examples use simple (de)serialization classes such as
>>> `LongDeseriailizer` where the name alone imparts all the information the
>>> user is likely to need. However, users may provide there own custom
>>> serialisers and such serialisers may contain state that is important, e.g.
>>> the serialiser may know the schema of the data being serialized.  May there
>>> be benefit from taking the `toString()` representation of the serialiser?
>>>
>>> Of course, this would require adding suitable `toString` impls to our own
>>> stock serialisers, but may ultimately prove more versatile in the future.
>>> The downside is potential to corrupt the topology description, e.g. a
>>> toString that includes new lines etc.
>>>
>>> Thanks,
>>>
>>> Andy
>>>
>>>
>>>
>>>> On 4 May 2020, at 19:27, Bruno Cadonna <br...@confluent.io> wrote:
>>>>
>>>> Hi Guozhang,
>>>>
>>>> Thank you for the KIP!
>>>>
>>>> Exposing also the inner types of the wrapper serdes would be
>>>> important. For debugging as Matthias has already mentioned and to see
>>>> more easily changes that are applied to a topology.
>>>>
>>>> I am also +1 on the `toJson()` method to easily access the topology
>>>> description programmatically and to make the description backward
>>>> compatible.
>>>>
>>>> Regarding `List<String> serdeNames();`, I would be in favour of a more
>>>> expressive return type, like a Map that assigns labels to Serde names.
>>>> For example, for key and value serdes the label could be "key" and
>>>> "value". Or something similar.
>>>>
>>>> Best,
>>>> Bruno
>>>>
>>>>
>>>>
>>>> On Mon, May 4, 2020 at 2:25 AM Guozhang Wang <wangg...@gmail.com>
>>> wrote:
>>>>>
>>>>> Hello Matthias John, thanks for your comments!! Replied them inline.
>>>>>
>>>>> I think there are a couple open questions that I'd like to hear your
>>>>> opinions on with the context:
>>>>>
>>>>> a. For stores's serdes, the reason I proposed to expose a set of serde
>>>>> names instead of a pair of key / value serdes is for future possible
>>> store
>>>>> types which may not be key-values. I admit it could just be
>>> over-killing
>>>>> here so if you have a strong preference on the latter, I could be
>>> convinced
>>>>> to change that part but I'd want to make the original motivation clear.
>>>>>
>>>>> b. I think I'm convinced that I'd just augment the `toString` result
>>>>> regardless of which func generated the Topology (and hence its
>>>>> TopologyDescription), note this would mean that we break the
>>> compatibility
>>>>> of the current `toString` function. As a remedy for that, we will also
>>>>> expose a `toJson` function to programmatical purposes.
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>>> (1) In the new TopologyDescription output, the line for the
>>>>>> windowed-count processor is:
>>>>>>
>>>>>>> Processor: myname (stores: [(myname-store, serdes:
>>>>> [SessionWindowedSerde, FullChangeSerde])])
>>>>>>
>>>>>> For this case, both Serdes are wrappers and user would actually only
>>>>>> specified wrapped Serdes for the key and value. Can we do anything
>>> about
>>>>>> this? Otherwise, there might still be a runtime `ClassCastException`
>>>>>> that a user cannot easily debug.
>>>>>>
>>>>>>
>>>>>> (2) Nit: The JavaDocs of `Processor#storeSet()` seems to be incorrect
>>>>>> (it says "The names of all connected stores." -- guess it's c&p
>>> error)?
>>>>>>
>>>>> Yes! Fixed.
>>>>>
>>>>>>
>>>>>> (3) The KIP mentioned to add `Store#changelogTopic()` method, but the
>>>>>> output of `TopologyDescription#toString()` does not contain it. I
>>> think
>>>>>> it might be good do add it, too?
>>>>>>
>>>>> Yes, that's right. I'm going to add to the example as well.
>>>>>
>>>>>>
>>>>>> (4) The KIP also list
>>> https://issues.apache.org/jira/browse/KAFKA-9913
>>>>>> but it seems not to address it yet?
>>>>>>
>>>>> I actually did intent to have it addressed; the proposal includes:
>>>>>
>>>>> a. Return the set of source / sink nodes of a sub-topology, and their
>>>>> corresponding source / sink topics could be accessed.
>>>>> b. Return the set of stores of a sub-topology, and their corresponding
>>>>> changelog topics could be accessed.
>>>>>
>>>>> The reason I did not choose to just expose the set of all topics
>>> directly,
>>>>> but indirectly, is stated in the wiki:
>>>>>
>>>>> "the reason we did not expose APIs for topic names directly is that for
>>>>> source nodes, it is possible to have Pattern and for sink nodes, it is
>>>>> possible to have topic-extractors, and hence it's better to let users
>>>>> leveraging on the lower-level APIs to construct the topic names
>>>>> programmatically themselves."
>>>>>
>>>>>>
>>>>>> (5) As John, I also noticed that `List<String> Store#sedersNames()` is
>>>>>> not a great API. I am not sure if I understand your reply thought.
>>>>>> AFAIK, there is no exiting API
>>>>>>
>>>>>>> List<Serde> StoreBuilder#serdes()
>>>>>>
>>>>>> (cf
>>>>>>
>>>>>
>>> https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/StoreBuilder.java
>>>>> )
>>>>>>
>>>>> Ah yes, that's added as part of this KIP.
>>>>>
>>>>>>
>>>>>> (6) Atm, we return `String` type for the Serdes. Do we think it's
>>>>>> sufficient? Just want to double check.
>>>>>
>>>>> The reason is that we can only get the serde-name at the time of the
>>>>> topology since it may be from config and hence there's no serde object
>>>>> actually.
>>>>>
>>>>>> (10) Can we avoid coupling this KIP’s behavior to the choice of
>>> ‘build’
>>>>> method? I.e., can we return the improved description even when people
>>> just
>>>>> call ‘build()’?
>>>>>
>>>>> Yes, as I replied in the above comment to yours, I've changed my mind
>>> to
>>>>> just return the augmented description no matter of the function; and
>>> will
>>>>> expose toJson() for future compatibilities. I've not yet updated the
>>> wiki
>>>>> yet.
>>>>>
>>>>>> Clearly, we need a placeholder if no serde is specified. How about
>>>>> “unknown”, or the name of the config keys,
>>>>> “default.key.serde”/“default.value.serde”?
>>>>>
>>>>> I think if `build(props)` is used, we can use the name of the
>>> configured
>>>>> values; otherwise since we do not know the config yet we'd have to use
>>>>> "unknown".
>>>>>
>>>>>> I still have some deep reservation about the ‘build(Parameters)’
>>> method
>>>>> itself. I don’t really want to side-track this conversation with all my
>>>>> concerns if we can avoid it, though. It seems like justification enough
>>>>> that introducing dramatically different behavior based in on seemingly
>>>>> minor differences in api calls will be a source of mystery and
>>> complexity
>>>>> for users.
>>>>>
>>>>>> I.e., I’m characterizing a completely different string format as
>>>>> “dramatically different”, as opposed to just having a placeholder
>>> string.
>>>>>
>>>>>> (11) Regarding the wrapper serdes, I bet we can capture and print the
>>>>> inner types as well.
>>>>>
>>>>> Ack, I can do that.
>>>>>
>>>>> On Sat, May 2, 2020 at 8:19 AM John Roesler <vvcep...@apache.org>
>>> wrote:
>>>>>
>>>>>> Hi all,
>>>>>>
>>>>>> I’ve been sitting on another concern about this proposal. Since
>>> Matthias
>>>>>> has just submitted a few questions, perhaps I can pile on two more
>>> this
>>>>>> round.
>>>>>>
>>>>>> (10) Can we avoid coupling this KIP’s behavior to the choice of
>>> ‘build’
>>>>>> method? I.e., can we return the improved description even when people
>>> just
>>>>>> call ‘build()’?
>>>>>>
>>>>>> Clearly, we need a placeholder if no serde is specified. How about
>>>>>> “unknown”, or the name of the config keys,
>>>>>> “default.key.serde”/“default.value.serde”?
>>>>>>
>>>>>> I still have some deep reservation about the ‘build(Parameters)’
>>> method
>>>>>> itself. I don’t really want to side-track this conversation with all
>>> my
>>>>>> concerns if we can avoid it, though. It seems like justification
>>> enough
>>>>>> that introducing dramatically different behavior based in on seemingly
>>>>>> minor differences in api calls will be a source of mystery and
>>> complexity
>>>>>> for users.
>>>>>>
>>>>>> I.e., I’m characterizing a completely different string format as
>>>>>> “dramatically different”, as opposed to just having a placeholder
>>> string.
>>>>>>
>>>>>> (11) Regarding the wrapper serdes, I bet we can capture and print the
>>>>>> inner types as well.
>>>>>>
>>>>>> Thanks again for the KIP!
>>>>>> -John
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Thu, Apr 30, 2020, at 19:10, Matthias J. Sax wrote:
>>>>>>> Guozhang,
>>>>>>>
>>>>>>> thanks for the KIP!
>>>>>>>
>>>>>>> Couple of comments/questions.
>>>>>>>
>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On 4/25/20 1:24 PM, Guozhang Wang wrote:
>>>>>>>> Hi John,
>>>>>>>>
>>>>>>>> Thanks for the review! Replied inline.
>>>>>>>>
>>>>>>>> On Fri, Apr 24, 2020 at 8:09 PM John Roesler <vvcep...@apache.org>
>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi Guozhang,
>>>>>>>>>
>>>>>>>>> Thanks for the KIP! I took a quick look, and I'm really happy to
>>> see
>>>>>> this
>>>>>>>>> underway.
>>>>>>>>>
>>>>>>>>> Some quick questions:
>>>>>>>>>
>>>>>>>>> 1.  Can you elaborate on the reason that stores just have a list of
>>>>>>>>> serdes, whereas
>>>>>>>>> other components have an explicit key/value serde?
>>>>>>>>>
>>>>>>>>
>>>>>>>> This is because of the existing API "List<Serde>
>>>>>> StoreBuilder#serdes()".
>>>>>>>> Although both of its implementations would return two serdes (one
>>> for
>>>>>> key
>>>>>>>> and one for value), the API is more general to return a list. And
>>>>>> hence the
>>>>>>>> TopologyDescription#Store which gets them directly from
>>> StoreBuilder is
>>>>>>>> exposing the same API.
>>>>>>>>
>>>>>>>> 1.5. A side-effect of this seems to be that the string-formatted
>>> serde
>>>>>>>>> description is
>>>>>>>>> different, depending on whether the serdes are listed on a store
>>> or a
>>>>>>>>> topic. Just an
>>>>>>>>> observation.
>>>>>>>>>
>>>>>>>>
>>>>>>>> Yes I agree. I think we can probably change the "List<Serde>
>>>>>>>> StoreBuilder#serdes()" signature as well (which would be a breaking
>>>>>> change
>>>>>>>> though, so we should do that via deprecation), but I'm a bit
>>> concerned
>>>>>>>> since it was designed for future store types which may not be of K-V
>>>>>> format
>>>>>>>> any more.
>>>>>>>>
>>>>>>>>
>>>>>>>>> 2. You mentioned the key compatibility concern in my mind. We do
>>> know
>>>>>> that
>>>>>>>>> such
>>>>>>>>> use cases exist. Namely, our own tests and
>>>>>>>>> https://zz85.github.io/kafka-streams-viz/
>>>>>>>>> I'm wondering if we can add a forward-compatible machine-readable
>>>>>> format
>>>>>>>>> to the
>>>>>>>>> KIP, so that even though we must break the parsers right now, maybe
>>>>>> we'll
>>>>>>>>> never
>>>>>>>>> have to break them again. For example, I'm thinking of a "toJson"
>>>>>> method
>>>>>>>>> on the
>>>>>>>>> TopologyDescription that formats the entire topology description
>>> as a
>>>>>> json
>>>>>>>>> string.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> Yes, I also have concerns about that (as described in the
>>> compatibility
>>>>>>>> section). One proposal I have is that we ONLY augment the toString
>>>>>> result
>>>>>>>> if the TopologyDescription is from a Topology built from
>>>>>>>> `StreamsBuilder#build(Properties)`, which is only recently added and
>>>>>> hence
>>>>>>>> most old usage would not get the benefits of it. But after thinking
>>>>>> about
>>>>>>>> this a bit more, I'm now more inclined to just always augment the
>>>>>> string,
>>>>>>>> and also add a `toJson` method in addition to `toString`.
>>>>>>>>
>>>>>>>>
>>>>>>>>> Thanks again!
>>>>>>>>> -John
>>>>>>>>>
>>>>>>>>> On Fri, Apr 24, 2020, at 00:26, Guozhang Wang wrote:
>>>>>>>>>> Hello folks,
>>>>>>>>>>
>>>>>>>>>> I'd like to start the discussion for KIP-598:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>
>>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=148648762
>>>>>>>>>>
>>>>>>>>>> It proposes to augment the topology description's sub-classes with
>>>>>> store
>>>>>>>>>> and source / sink serde information. Let me know what you think,
>>>>>> thanks!
>>>>>>>>>>
>>>>>>>>>> --
>>>>>>>>>> -- Guozhang
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> Attachments:
>>>>>>> * signature.asc
>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> -- Guozhang
>>>
>>>
>>
>> --
>> -- Guozhang
>>
> 
> 

Attachment: signature.asc
Description: OpenPGP digital signature

Reply via email to