What is the status of this KIP?

I also want to point out, that it seems that there is some overlap with
KIP-307 that was proposed recently (I cc'ed Florian who proposed
KIP-307). KIP-230 might even be subsumed.

I any case, it would be good to sync up and collaborate for both KIPs.


-Matthias


On 2/28/18 1:07 PM, Guozhang Wang wrote:
> Hi Matthias,
> 
> I've also made a pass over the KIP, aside from the-other-Matthias's
> comment, I'm wondering if you have scenarios that want to distinguish the
> two internal topics of the join?
> 
> Currently we use "-this" and "-other" suffix for the topics. So for example:
> 
> stream1.join(stream2, ...)  // stream1 will be materialized with "-this",
> and stream2 with "-other"
> 
> While:
> 
> stream2.join(stream1, ...)  // stream2 will be materialized with "-this",
> and stream1 with "-other"
> 
> 
> If we think it is reasonable to require users be aware that the above join
> situations are not exactly the same, then the current naming is fine; if we
> want them to be mutually reused (I'm not sure if this is a common case?)
> then we probably need to consider something new?
> 
> 
> 
> Guozhang
> 
> 
> 
> 
> 
> On Tue, Feb 13, 2018 at 7:02 PM, Matthias Margush <
> matthias.marg...@gmail.com> wrote:
> 
>> Thanks for the reminder! I need to do some wordsmithing based on the
>> feedback I’ve gotten. I’ll do that soon (hopefully).
>> On Tue, Feb 13, 2018 at 1:45 PM Matthias J. Sax <matth...@confluent.io>
>> wrote:
>>
>>> Is there any updates for this KIP?
>>>
>>> -Matthias
>>>
>>> On 12/28/17 12:27 PM, Matthias J. Sax wrote:
>>>> Thanks for updating the KIP.
>>>>
>>>> The code-diff is a  little hard to read. It's better to so something
>>>> similar as in this KIP:
>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 245%3A+Use+Properties+instead+of+StreamsConfig+in+KafkaStreams+constructor
>>>>
>>>> (Just as an example. Maybe take a look as other KIPs, too.)
>>>>
>>>> Some side remarks:
>>>>  - please update the link to the DISCUSS thread
>>>>  - there are some typos: Kstream -> KStream; Topology Builder exception
>>>> -> TopologyBuilderException
>>>>
>>>>
>>>> You propose to add `otherValueSerde(final String joinName)` -- I guess
>>>> the method name is a c&p error and method name must be updated?
>>>>
>>>> Changes to internal classes like `KStreamImpl` are not required in the
>>>> KIP as those as implementation details. The KIP should focus on public
>>>> changes.
>>>>
>>>>
>>>> -Matthias
>>>>
>>>>
>>>> On 12/26/17 11:19 AM, Matthias Margush wrote:
>>>>> Greetings.
>>>>>
>>>>> Thanks for the comments and suggestions. I updated the KIP with these
>>>>> proposals for the questions posed by Matt & Matthias:
>>>>>
>>>>> *Can you please c&p the corresponding content instead of just
>>>>> putting links? A KIP should be a self-contained Wiki page. Also, if we
>>> add
>>>>> a optional config parameter, how would we specify it? **Please list
>> all
>>>>> changes to want to apply to `Joined` class.*
>>>>>
>>>>> I added more details around the proposed changes directly to the KIP.
>>>>>
>>>>> *I will point out that your KIP doesn't outline what would happen if
>>>>> you picked a name that resulted in a non unique topic name? What would
>>> be
>>>>> the error handling behavior there?*
>>>>>
>>>>> Looking at the current behavior of methods that allow the user to
>>> specify
>>>>> names for internal resources (e.g. `reduce`, `aggregate`), I added a
>>>>> proposal that the code generate a similar exception if a name conflict
>>> is
>>>>> detected in the topology:
>>>>>
>>>>> org.apache.kafka.streams.errors.TopologyBuilderException: "Invalid
>>> topology
>>>>> building: Topic reduction-same-name-repartition has already been
>>> registered
>>>>> by another source."
>>>>>
>>>>> *What is the impact on KStream-KTable join?*
>>>>>
>>>>> Proposed that kstream-ktable joins similarly make use of the provided
>>>>> joinName when generating internal repartition topics.
>>>>>
>>>>> On Mon, Dec 4, 2017 at 2:57 PM Matthias J. Sax <matth...@confluent.io
>>>
>>>>> wrote:
>>>>>
>>>>>> Matthias,
>>>>>>
>>>>>> thanks for the KIP.
>>>>>>
>>>>>> Can you please c&p the corresponding content instead of just putting
>>>>>> links? A KIP should be a self-contained Wiki page.
>>>>>>
>>>>>> Also, if we add a optional config parameter, how would we specify it?
>>>>>> Please list all changes to want to apply to `Joined` class.
>>>>>>
>>>>>> Furthermore, `Joined` is also used for KStream-KTable join but the
>> KIP
>>>>>> only talks about windowed joins (ie, KStream-KTream join). What the
>>>>>> impact on KStream-KTable join?
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 11/29/17 6:09 AM, Matt Farmer wrote:
>>>>>>> Hi Matthias,
>>>>>>>
>>>>>>> I certainly have found the auto-generated names unwieldy while doing
>>>>>>> cluster administration.
>>>>>>>
>>>>>>> I will point out that your KIP doesn't outline what would happen if
>>> you
>>>>>>> picked a name that resulted in a non unique topic name? What would
>> be
>>> the
>>>>>>> error handling behavior there?
>>>>>>>
>>>>>>> On Wed, Nov 29, 2017 at 9:03 AM Matthias Margush <
>>>>>> matthias.marg...@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi everyone,
>>>>>>>>
>>>>>>>> I created this KIP to allow windowing joins to be named. If named,
>>> then
>>>>>> the
>>>>>>>> associated internal topic names would be derived from that, instead
>>> of
>>>>>>>> being randomly generated.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP+
>> 230%3A+Name+Windowing+Joins
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>>
>>>>>>>> Matthias
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>>
>>
> 
> 
> 

Attachment: signature.asc
Description: OpenPGP digital signature

Reply via email to