Re: Kafka Streams And Partitioning

2021-03-20 Thread Gareth Collins
Hi Sophie, I will have to give this a try. Thanks very much! Gareth On Fri, Mar 19, 2021 at 9:00 PM Sophie Blee-Goldman wrote: > Ah ok, I think I was envisioning a different use case from your initial > description of the problem. > If everything that you want to group together is already corr

Re: Kafka Streams And Partitioning

2021-03-19 Thread Sophie Blee-Goldman
Ah ok, I think I was envisioning a different use case from your initial description of the problem. If everything that you want to group together is already correctly partitioned, then you won't need a repartitioning step. If I understand correctly, you have something like this in mind: builder

Re: Kafka Streams And Partitioning

2021-03-17 Thread Gareth Collins
Hi Sophie, Thanks very much for the response! So if I understand correctly it will be impossible to avoid the repartition topic? e.g. my original message may have key = A...and will be partitioned on A. But in my Kafka Streams app, I will want to aggregate on A:B or A:C or A:D (B, C or D come f

Re: Kafka Streams And Partitioning

2021-03-17 Thread Sophie Blee-Goldman
Hey Gareth, Kafka Streams state store partitioning is based on the partitioning of the upstream input topics. If you want your RocksDB stores to be partitioned based on the prefix of a key, then you should make sure the input topic feeding into it uses whatever partitioning strategy you had in min

Kafka Streams And Partitioning

2021-03-15 Thread Gareth Collins
Hi, This may be a newbie question but is it possible to control the partitioning of a RocksDB KeyValueStore in Kafka Streams? For example, I perhaps only want to partition based on a prefix of a key rather than the full key. I assume something similar must be done for the WindowStore to partition

Re: Joins in Kafka Streams and partitioning of the topics

2017-11-30 Thread Artur Mrozowski
Yes you are probably right. So I was inspired be the KIP 150 blog post, so the entire statement would be like this: KTable customerGrouped= kStreamBuilder.stream(stringSerde, customerMessageSerde, CUSTOMER_TOPIC) .groupBy((key,value) -> Integer.parseInt(value.customer.replaceFirs

Re: Joins in Kafka Streams and partitioning of the topics

2017-11-30 Thread Jan Filipiak
There are some oddities in your topology that make make we wonder if they are the true drivers of your question. https://github.com/afuyo/KStreamsDemo/blob/master/src/main/java/kstream.demo/CustomerStreamPipelineHDI.java#L300 Feels like it should be a KTable to begin with for example otherwise i

Re: Joins in Kafka Streams and partitioning of the topics

2017-11-30 Thread Artur Mrozowski
what if I start two instances of that application? Does the state migrate between the applications? Is it then I have to use a global table? BR Artur On Thu, Nov 30, 2017 at 7:40 PM, Jan Filipiak wrote: > Hi, > > Haven't checked your code. But from what you describe you should be fine. > Upgra

Re: Joins in Kafka Streams and partitioning of the topics

2017-11-30 Thread Jan Filipiak
Hi, Haven't checked your code. But from what you describe you should be fine. Upgrading the version might help here and there but should still work with 0.10 I guess. Best Jan On 30.11.2017 19:16, Artur Mrozowski wrote: Thank you Damian, it was very helpful. I have implemented my solution i

Re: Joins in Kafka Streams and partitioning of the topics

2017-11-30 Thread Artur Mrozowski
Thank you Damian, it was very helpful. I have implemented my solution in version 0.11.0.2 but there is one thing I still wonder. So what I try to do is what is described in KIP 150. Since it didn't make to the release for 1.0 I do it the old fashioned way. https://cwiki.apache.org/confluence/displa

Re: Joins in Kafka Streams and partitioning of the topics

2017-11-22 Thread Damian Guy
Hi Artur, KafkaStreams 0.10.0.0 is quite old and a lot has changed and been fixed since then. If possible i'd recommend upgrading to at least 0.11.0.2 or 1.0. For joins you need to ensure that the topics have the same number of partitions (which they do) and that they are keyed the same. Thanks,

Joins in Kafka Streams and partitioning of the topics

2017-11-22 Thread Artur Mrozowski
Hi, I am joining 4 different topic with 4 partitions each using 0.10.0.0 version of Kafka Streams. The joins are KTable to KTable. Is there anything I should be aware of considering partitions or version of Kafka Streams? In other words should I be expecting consistent results or do I need to for