Re: [VOTE] KIP-349 Priorities for Source Topics

2019-01-06 Thread Adam Bellemare
Hi Nick Is this change only for the basic consumer? How would this affect anything with Kafka Streams? Thanks > On Jan 5, 2019, at 10:52 PM, n...@afshartous.com wrote: > > Bumping again for more votes. > -- > Nick > > >> On Dec 26, 2018, at 12:36 PM, n...@afshartous.com wrote: >>

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2019-01-02 Thread Adam Bellemare
inion on this. For me its dead, I just leave you the > message here as an opportunity to reconsider the choices that were made. > > Whish y'll a happy new year :) > > > > > > > On 27.12.2018 17:22, Adam Bellemare wrote: > > Hi All > > > > Sorry for the delay - ho

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-12-27 Thread Adam Bellemare
, 2018 at 7:06 PM Adam Bellemare wrote: > Hi John and Guozhang > > Ah yes, I lost that in the mix! Thanks for the convergent solutions - I do > think that the attachment that John included makes for a better design. It > should also help with overall performance as very high-cardinality

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-12-17 Thread Adam Bellemare
sumption, which we can > > discuss more if you're suspicious. > > * we only need one copy of the data (the left and right tables need to be > > materialized) and one extra copy of the PK:FK pairs in the Materialized > > Subscription Store. Materializing the join result is option

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-12-17 Thread Adam Bellemare
gt; enduring, and which parts are a snapshot of our implementation thinking > at > > the time. > > > > I'm suggesting this because I suspect that the others haven't made time > to > > review it partly because it seems daunting. If it seems like it would be >

Re: Vote for KIP-393 (Fix time windowed serde to deserialize changelog topic)

2018-12-12 Thread Adam Bellemare
+1 (non-binding) from me. Looks like a pretty clear-cut case. On Tue, Dec 11, 2018 at 1:11 AM Shawn Nguyen wrote: > Thanks for the feedback Guozhang! I updated the KIP. > > In the meantime, could I ask for additional binding votes/approval on this > KIP proposal? > > Shawn > > On Thu, Dec 6,

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-12-10 Thread Adam Bellemare
> > we > > > can just keep a single final-result store with timestamps and reject > > values > > > that have a smaller timestamp, is that right? > > > > Which is the correct output should at least be decided on the offset of > > the original message. &

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-12-04 Thread Adam Bellemare
ded as-is. Alternate option is possible, but will require for implementation details to be made in the API and some exposure of new data structures into the API (ie: CombinedKey). I appreciate any insight into this. Thanks. On Tue, Dec 4, 2018 at 2:59 PM Adam Bellemare wrote: > Hi John > > Tha

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-12-04 Thread Adam Bellemare
a KTable, and then do things like > join, re-group, etc from there... I'm not sure; I haven't thought through > all the consequences yet. > > This is all just my opinion after thinking over the discussion so far... > -John > > On Mon, Dec 3, 2018 at 2:56 PM Adam Bellemare > wrote:

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-11-30 Thread Adam Bellemare
oblem of large values, as > > there's no bound on the number of lhs records that can reference rhs > > records. Offhand, I'd say we could page the values, so when one row is > past > > the threshold, we append the key for the next page. Then in most cases, > it > > would

Re: [DISCUSS] KIP-258: Allow to Store Record Timestamps in RocksDB

2018-11-22 Thread Adam Bellemare
> > This would leave the original store intact and allow a migration of the > > timestamps without touching the other data. > > > > So I am very interested in your PR :-) > > > > best regards > > > > Patrik > > > > On Tue, 20 Nov 2018 at 04:46

Re: [DISCUSS] KIP-258: Allow to Store Record Timestamps in RocksDB

2018-11-19 Thread Adam Bellemare
ith some colleagues and I am prototyping this atm. Hope to > update the KIP accordingly soon. > > -Matthias > > On 11/10/18 7:41 AM, Adam Bellemare wrote: > > Hello Matthias > > > > I am curious as to the status of this KIP. TTL and expiry of records will > >

Re: [DISCUSS] KIP-258: Allow to Store Record Timestamps in RocksDB

2018-11-10 Thread Adam Bellemare
Hello Matthias I am curious as to the status of this KIP. TTL and expiry of records will be extremely useful for several of our business use-cases, as well as another KIP I had been working on. Thanks On Mon, Aug 13, 2018 at 10:29 AM Eno Thereska wrote: > Hi Matthias, > > Good stuff. Could

Re: [VOTE] - KIP-213 Support non-key joining in KTable

2018-11-08 Thread Adam Bellemare
over again. Would it be discouraging if I wound up responding to your > discussion thread? > > Thanks, > -John > > On Wed, Nov 7, 2018 at 3:25 PM Adam Bellemare > wrote: > >> Bumping this thread, as per convention - 1 >> >> On Fri, Nov 2, 2018 at 8:22 AM Adam B

Re: [VOTE] - KIP-213 Support non-key joining in KTable

2018-11-07 Thread Adam Bellemare
Bumping this thread, as per convention - 1 On Fri, Nov 2, 2018 at 8:22 AM Adam Bellemare wrote: > As expected :) But still, thanks none-the-less! > > On Fri, Nov 2, 2018 at 3:36 AM Jan Filipiak > wrote: > >> reminder >> >> On 30.10.2018 15:47, Adam Bellema

Re: [VOTE] - KIP-213 Support non-key joining in KTable

2018-11-02 Thread Adam Bellemare
As expected :) But still, thanks none-the-less! On Fri, Nov 2, 2018 at 3:36 AM Jan Filipiak wrote: > reminder > > On 30.10.2018 15:47, Adam Bellemare wrote: > > Hi All > > > > I would like to call a vote on > > > https://cwiki.apache.org/confluence/display/K

[VOTE] - KIP-213 Support non-key joining in KTable

2018-10-30 Thread Adam Bellemare
Hi All I would like to call a vote on https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable. This allows a Kafka Streams DSL user to perform KTable to KTable foreign-key joins on their data. I have been using this in production for some time and I have

[jira] [Created] (KAFKA-7552) StatefulProcessorNode tries to connect state store to processor before it is added

2018-10-25 Thread Adam Bellemare (JIRA)
Adam Bellemare created KAFKA-7552: - Summary: StatefulProcessorNode tries to connect state store to processor before it is added Key: KAFKA-7552 URL: https://issues.apache.org/jira/browse/KAFKA-7552

Re: Design patterns in kafka

2018-10-20 Thread Adam Bellemare
I think this question is much too broad to be answered in this way. This will require you to do your own work on understanding the code. Alternately, start by looking through the Kafka documentation and Confluent blogs to get an understanding of Kafka usage patterns in Event driven

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-10-09 Thread Adam Bellemare
? Thanks Adam On Sun, Sep 30, 2018 at 10:23 AM Adam Bellemare wrote: > Hi Jan > > With the Stores.windowStoreBuilder and Stores.persistentWindowStore, you > actually only need to specify the amount of segments you want and how large > they are. To the best of my understandin

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-09-30 Thread Adam Bellemare
> > I just hope the window-store doesn't check stream-time under the hoods > that would be a questionable interface. > > If it does: did you see my comment on checking all the windows earlier? > that would be needed to actually give reasonable time gurantees. > > Best >

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-09-25 Thread Adam Bellemare
Hi Jan Check for " highwaterMat " in the PR. I only changed the state store, not the ProcessorSupplier. Thanks, Adam On Mon, Sep 24, 2018 at 2:47 PM, Jan Filipiak wrote: > > > On 24.09.2018 16:26, Adam Bellemare wrote: > >> @Guozhang >> >> Thanks for t

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-09-24 Thread Adam Bellemare
, but for windowed KTables we do not really have a good support for > > their joins anyways (https://issues.apache.org/jira/browse/KAFKA-7107) I > > think we can just consider non-windowed KTable-KTable non-key joins for > > now. In which case, KIP-258 should help. > &g

Re: A question about kafka streams API

2018-09-12 Thread Adam Bellemare
on an invalid message? and not consuming any further messages? > > Thanks again > > On Wed, Sep 12, 2018 at 2:35 PM Adam Bellemare > wrote: > > > Hi Yui Yoi > > > > Preface: I am not familiar with the spring framework. > > > > "Earliest"

Re: A question about kafka streams API

2018-09-12 Thread Adam Bellemare
Hi Yui Yoi Preface: I am not familiar with the spring framework. "Earliest" when it comes to consuming from Kafka means, "Start reading from the first message in the topic, *if there is no offset stored for that consumer group*". It sounds like you are expecting it to re-read each message

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-09-11 Thread Adam Bellemare
your charts. > > Hopefully i can get it done today > > > > On 08.09.2018 16:00, Adam Bellemare wrote: > > > Hi Jan > > > > > > I have included a diagram of what I attempted on the KIP. > > > > > https://cwiki.apache.org/confluence/display/

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-09-08 Thread Adam Bellemare
e somewhere >> I can see the original requirement or proposal? >> >> On Sep 7, 2018, at 8:13 AM, Jan Filipiak >>> wrote: >>> >>> >>> On 05.09.2018 22:17, Adam Bellemare wrote: >>> >>>> I'm currently testing using a Windo

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-09-05 Thread Adam Bellemare
ing else afterwards is really common. > > > > > > > On 05.09.2018 13:49, Adam Bellemare wrote: > >> Hi Matthias >> >> Thank you for your feedback, I do appreciate it! >> >> While name spacing would be possible, it would require to deserialize >>> user headers

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-09-05 Thread Adam Bellemare
d. The decision about the forward > > depends on the size of the aggregated map. > > Only 1 element long maps would be unpacked and forwarded. 0 element maps > > would be published as delete. Any other count > > of map entries is in "waiting for correct deletes to ar

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-09-04 Thread Adam Bellemare
...). This in effect is the same as the design I have now, just with the two tables merged together. I will keep looking at this but I am not seeing a great simplification. Advice and comments are welcomed as always. On Tue, Sep 4, 2018 at 9:38 AM, Adam Bellemare wrote: > > As I was lookin

Re: [VOTE] KIP-357: Add support to list ACLs per principal

2018-09-04 Thread Adam Bellemare
+1 (non binding) - would really like to see this one. On Mon, Sep 3, 2018 at 12:18 PM, Mickael Maison wrote: > +1 (non binding) > On Mon, Sep 3, 2018 at 3:14 PM Manikumar > wrote: > > > > bump up! waiting for 2 more binding votes! > > > > On Tue, Aug 28, 2018 at 7:36 AM Satish Duggana > > >

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-09-04 Thread Adam Bellemare
on this are appreciated. I will revisit it again when I have a bit more time. Thanks On Mon, Sep 3, 2018 at 4:55 PM, Adam Bellemare wrote: > Hi Jan > > Thank you for taking the time to look into my PR. I have updated it > accordingly along with the suggestions from John. Please no

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-09-03 Thread Adam Bellemare
iew and further KIP discussion. I can >> understand this can be painful, but that may be the best option we can do >> to avoid as much work to be wasted as possible. >> >> >> Guozhang >> >> >> On Wed, Aug 29, 2018 at 10:06 AM, Adam Bellemare

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-08-29 Thread Adam Bellemare
y for reviews? > > > Guozhang > > On Tue, Aug 28, 2018 at 5:00 AM, Adam Bellemare > wrote: > >> Okay, I will implement John's suggestion of namespacing the external >> headers prior to processing, and then removing the namespacing prior to >> emitting. A pote

Re: [VOTE] KIP-275 - Indicate "isClosing" in the SinkTaskContext

2018-08-29 Thread Adam Bellemare
Hi Matt I posted in the PR about some of the commenting changes. Overall I think the whole thing looks good, so I would give it a: +1 non-binding Adam On Tue, Aug 28, 2018 at 5:13 PM, Matt Farmer wrote: > Given that voting and discussion have stalled out it seems like this is a > thing that

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-08-28 Thread Adam Bellemare
he header to be safe, while keeping the additional cost > (note the header field is per-record, so any additional byte is per-record > as well) low. > > > Guozhang > > On Tue, Aug 21, 2018 at 11:58 AM, Adam Bellemare > > wrote: > > > Hi John > > > > Th

Re: Current Kafka Steams and KSQL Performance Metrics / Benchmarks?

2018-08-23 Thread Adam Bellemare
lable%22)% > 20AND%20component%20%3D%20%22streams%22%20%20AND% > 20labels%20%3D%20performance%20%20 > > > Guozhang > > On Wed, Aug 22, 2018 at 7:02 AM, Adam Bellemare > wrote: > > > Blog post in question: > > https://www.confluent.io/blog/ksql-february-release

Re: Current Kafka Steams and KSQL Performance Metrics / Benchmarks?

2018-08-22 Thread Adam Bellemare
Blog post in question: https://www.confluent.io/blog/ksql-february-release-streaming-sql-for-apache-kafka/ On Wed, Aug 22, 2018 at 10:01 AM, Adam Bellemare wrote: > Hi All > > I am looking for performance metrics related to Kafka Streams and KSQL. I > have been scouring various blog

Current Kafka Steams and KSQL Performance Metrics / Benchmarks?

2018-08-22 Thread Adam Bellemare
Hi All I am looking for performance metrics related to Kafka Streams and KSQL. I have been scouring various blogs, including the confluent one, looking for any current performance metrics or benchmarks, official or otherwise, on both Kafka Streams and KSQL for Kafka 2.x +. Unfortunately, almost

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-08-21 Thread Adam Bellemare
t 9:48 AM Jan Filipiak > wrote: > > > Still havent completly grabbed it. > > sorry will read more > > > > On 17.08.2018 21:23, Jan Filipiak wrote: > > > Cool stuff. > > > > > > I made some random remarks. Did not touch the core of

Re: [VOTE] KIP-349 Priorities for Source Topics

2018-08-19 Thread Adam Bellemare
While I am not sure if I can or can’t vote, my question re: Jan’s comment is, “should we be implementing it as Samza does?” I am not familiar with the drawbacks of the current approach vs how samza does it. > On Aug 18, 2018, at 5:06 PM, n...@afshartous.com wrote: > > > I only saw one vote

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-08-17 Thread Adam Bellemare
gher > streams version, I don't really have a steak here and initially I didn't > feel like actually sending it. But maybe you can pull > something good from it. > > Best jan > > > > On 15.08.2018 04:44, Adam Bellemare wrote: > >> @Jan >> Thanks Jan.

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-08-14 Thread Adam Bellemare
ea > > of hiding this behind metadata and handle it in the DSL is from my POV > > unideal. > > > > To write into your example: > > > > key + A, null) > > (key +B, ) > > > > is what my output would look like. > > > > > > Hope that make

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-08-13 Thread Adam Bellemare
CC Jan On Mon, Aug 13, 2018 at 12:16 PM, Adam Bellemare wrote: > Hi Jan > > If you do not use headers or other metadata, how do you ensure that > changes to the foreign-key value are not resolved out-of-order? > ie: If an event has FK = A, but you change it to FK = B, you need

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-08-13 Thread Adam Bellemare
is anything unclear with all the text that has been written, feel > free to just directly cc me so I don't miss it on > the mailing list. > > Best Jan > > > > > > On 08.08.2018 15:26, Adam Bellemare wrote: > >> More followup, and +dev as Guozhang replied to me directly pre

Re: Kafka stream - Internal topic name and schema avro compatibility

2018-08-09 Thread Adam Bellemare
We have been discussing various mechanisms by which we could support > > modifying the topology in place. Typically, this would involve giving > each > > operator a semantic name so that the internal names would be related to > > what the nodes are doing, not the ord

Re: Kafka stream - Internal topic name and schema avro compatibility

2018-08-08 Thread Adam Bellemare
Hi Cédric I do not know how the topology names are chosen, but provided that you didn't change any of the topology then new topics will not be created or require alteration. If you modify the topology then the naming can indeed change, but it would then create a new internal topic and there

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-08-08 Thread Adam Bellemare
will be switching the KIP-213 to use the RecordHeaders in Kafka Streams instead of the PropagationWrapper, but conceptually it should be the same. Again, any feedback is welcomed... On Mon, Jul 30, 2018 at 9:38 AM, Adam Bellemare wrote: > Hi Guozhang et al > > I was just reading the 2.0 rele

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-07-30 Thread Adam Bellemare
data in streams. Adam On Fri, Jul 27, 2018 at 1:45 AM, Guozhang Wang wrote: > Hello Adam, > > Thanks for rebooting the discussion of this KIP ! Let me finish my pass on > the wiki and get back to you soon. Sorry for the delays.. > > Guozhang > > On Tue, Jul 24, 2018 at

Re: Processor API StateStore and Recovery with State Machines question.

2018-07-26 Thread Adam Bellemare
;. > > 3) Finally, when exactly-once is enabled, if there is any crashes, the > changelog topic / state store will be "rewinded" (I omit the implementation > details here, but just assume that logically, we can rewind them) to the > previously successful commit, so `exactly-once` i

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-07-24 Thread Adam Bellemare
so desire, but first I was hoping to just tackle some of the fundamental design proposals. Thanks, Adam On Mon, Jul 23, 2018 at 10:05 AM, Adam Bellemare wrote: > Here is the new discussion thread for KIP-213. I picked back up on the KIP > as this is something that we too at Flipp are now runni

KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-07-23 Thread Adam Bellemare
clearer integration tests (I did a lot of testing using a non-Kafka Streams framework) and clean up the code a bit more before putting it in a PR against trunk (I can do so later this week likely). Please take a look, Thanks Adam Bellemare

Processor API StateStore and Recovery with State Machines question.

2018-07-22 Thread Adam Bellemare
Hi Folks I have a quick question about a scenario that I would appreciate some insight on. This is related to a KIP I am working on, but I wanted to break this out into its own scenario to reach a wider audience. In this scenario, I am using builder.internalTopologyBuilder to create the following

Restarting KIP-213 discussion

2018-07-17 Thread Adam Bellemare
Hey all I recently took a crack at implementing my own version of KIP-213 based on code written by the previous committee, Jan. I currently have it dev ready, but I want to restart some of the discussion based on some design decisions that I made, and I need some feedback on a few particular

Re: [DISCUSS] - KIP-314: KTable to GlobalKTable Bi-directional Join

2018-06-25 Thread Adam Bellemare
t; Please see my comments inline. > > On Thu, Jun 21, 2018 at 8:14 AM, Adam Bellemare > wrote: > > > Hi Guozhang > > > > *Re: Questions* > > *1)* I do not yet have a solution to this, but I also did not look that > > closely at it when I begun this K

Re: [DISCUSS] - KIP-314: KTable to GlobalKTable Bi-directional Join

2018-06-21 Thread Adam Bellemare
uter, while for > KStream-KTable / GlobalKTable join we only have inner / left, and the > reason is that for stream-table joins outer join makes less sense; should > we consider outer for KTable-GlobalKTable join as well? > > > Guozhang > > > On Tue, Jun 19, 2018 at 1

Re: [DISCUSS] - KIP-314: KTable to GlobalKTable Bi-directional Join

2018-06-19 Thread Adam Bellemare
t; > It suggest to add non-key joins and a lot of issues how to implement > this were discussed already. As a KTable-GloblKTable join is a non-key > join, too, it seems that those discussion apply to your KIP too. > > Hope this helps to make the next steps. > > > -Matthias

[DISCUSS] - KIP-314: KTable to GlobalKTable Bi-directional Join

2018-06-18 Thread Adam Bellemare
alter the way that GlobalKTables can be used in relation to KTables. I believe that this would be a very useful change but I need some eyes on the technical aspects to validate or refute the strategy. Thanks Adam Bellemare

Re: Need Access to Create KIP - Second Time Requesting

2018-06-09 Thread Adam Bellemare
wrote: > Your request was already granted. > > Same day when you sent the first email. > > Did you not see my reply on the dev list? (cc'ed you this time to make > sure you get the email). > > > -Matthias > > On 6/8/18 5:05 AM, Adam Bellemare wrote: > > Hello

Need Access to Create KIP - Second Time Requesting

2018-06-08 Thread Adam Bellemare
Hello Sending a second request to get access to make a KIP. As per: https://cwiki.apache.org/confluence/display/KAFKA/ Kafka+Improvement+Proposals I request access to be able to create a KIP for https://issues.apache.org/ jira/browse/KAFKA-4628. Wiki username: adam.bellemare Thanks, Adam

Request Access to Create KIP

2018-06-06 Thread Adam Bellemare
Hello As per: https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals I request access to be able to create a KIP for https://issues.apache.org/jira/browse/KAFKA-4628. Wiki username: adam.bellemare Thanks, Adam

<    1   2