Re: outerjoin not joining after window

2024-05-21 Thread Chad Preisler
is written to the partition after the window closes? On Tue, May 21, 2024 at 10:27 AM Chad Preisler wrote: > See one small edit below... > > On Tue, May 21, 2024 at 10:25 AM Chad Preisler > wrote: > >> Hello, >> >> I think the issue is related to certa

Re: outerjoin not joining after window

2024-05-21 Thread Chad Preisler
See one small edit below... On Tue, May 21, 2024 at 10:25 AM Chad Preisler wrote: > Hello, > > I think the issue is related to certain partitions not getting records to > advance stream time (because of low volume). Can someone confirm that each > partition has its

Re: outerjoin not joining after window

2024-05-21 Thread Chad Preisler
e dropped right away. So you cannot really infer that a record > did make it into the join or not, or what happens if it did make it into > the `Processor`. > > -> https://kafka.apache.org/documentation/#kafka_streams_task_monitoring > > `dropped-records-total` is the name of the metr

Re: outerjoin not joining after window

2024-05-01 Thread Chad Preisler
t; malformed" records that is also recorded via JMX. > > > WARN: "Skipping record due to null key or value. " > > > If you can identify a specific record from the input which would produce > an output, but does not, maybe you can try to feed it into your

Re: outerjoin not joining after window

2024-04-30 Thread Chad Preisler
send new records, "stream-time" will stop to advance and the result > might not be emitted because the window does not get closed. > > (Last, there is some internal wall-clock time delay of one second to > emit results for performance reasons...) > > HTH. > > -M

outerjoin not joining after window

2024-04-30 Thread Chad Preisler
Hello, I have a KStream to KStream outer join with a time difference of 25 minutes and 5 minutes of grace. When I get a record for one side of the join, but don't get a record on the other side of the join, I expect the join to execute after the 25 with one side of the join containing a record

Re: outerJoin confusion

2024-04-05 Thread Chad Preisler
> Maybe we should disable this config when topology test driver is used > automatically... It's not the first time it did came up. > > I opened a PR for it: https://github.com/apache/kafka/pull/15660 > > > -Matthias > > > > On 4/3/24 3:52 PM, Chad Preisler wrote:

Re: outerJoin confusion

2024-04-03 Thread Chad Preisler
me-java.time.Duration- > > > Regards, > Shashwat Pandey > > > On Wed, Apr 3, 2024 at 5:05 PM Chad Preisler > wrote: > > > Seems like there is some issue with the TopologyTestDriver. I am able to > > run the same stream against Kakfa and I'm gettin

Re: outerJoin confusion

2024-04-03 Thread Chad Preisler
of join? On Wed, Apr 3, 2024 at 2:46 PM Chad Preisler wrote: > Hello, > > I'm confused about the outerJoin and when records are produced with the > following code. > > Topology buildTopology() { > var builder = new StreamsBuilder(); > var leftStream = builder.

outerJoin confusion

2024-04-03 Thread Chad Preisler
Hello, I'm confused about the outerJoin and when records are produced with the following code. Topology buildTopology() { var builder = new StreamsBuilder(); var leftStream = builder.stream("leftSecondsTopic", Consumed.with(Serdes.String(), Serdes.String())); var

Re: leftjoin not working as expected.

2022-08-10 Thread Chad Preisler
that happens the join code should execute with the right side as > "null". So my question is: when you see that join func executed with the > left side as the first record, is the right side "null"? If yes I think > that's reflecting what I'm describing here. > >

leftjoin not working as expected.

2022-08-04 Thread Chad Preisler
Hello, I'm doing a stream to stream leftjoin. Here is what I am seeing when I test the code. - I write a record to the left side topic. The stream app reads the message and the deserializer gets triggered. However, the join is not triggered at this time. - I write another record to the left

Re: Kafka streams usecase

2022-02-17 Thread Chad Preisler
d in message which is the key > in postgress database . Can this be done with a simple consumer ? > > On Thu, Jan 13, 2022 at 11:11 AM Chad Preisler > wrote: > > > Yes Kafka streams can be used to do this. There are probably several ways > > to implement this. W

Re: Kafka Consumer Fairness when fetching events from different partitions.

2022-01-31 Thread Chad Preisler
Hello, I got this from the JavaDocs for KafkaConsumer. * If a consumer is assigned multiple partitions to fetch data from, it will try to consume from all of them at the same time, * effectively giving these partitions the same priority for consumption. However in some cases consumers may want

Timestamps unique?

2022-01-13 Thread Chad Preisler
Hello, For ConsumerRecord.timestamp() is the timestamp guaranteed to be unique within the topic's partition, or can there be records inside the topics partition that have the same timestamp? Thanks. Chad

Re: Kafka streams usecase

2022-01-13 Thread Chad Preisler
Yes Kafka streams can be used to do this. There are probably several ways to implement this. We did something like this in Java using a groupByKey() and reduce() functions. The three topics we wanted to combine into one topic had different schemas and different java class types. So to combine them

Re: KTable updates by non-key field

2021-11-19 Thread Chad Preisler
th just the IDs this should not be > a problem. > > Do you have examples how you implemented this stuff with the processor API? > > Best, > Claudia > > > -----Ursprüngliche Nachricht- > Von: Chad Preisler > Gesendet: Donnerstag, 18. November 2021 20:48 > An: users@ka

Re: Stream to KTable internals

2021-11-18 Thread Chad Preisler
ata, to better > understand the underlying behavior. > > > -Matthias > > On 11/5/21 6:58 AM, Chad Preisler wrote: > > It seems like I have 2 options to work around this issue. > > > > > > - Keep the KTable and have another process running that puts the >

Re: KTable updates by non-key field

2021-11-18 Thread Chad Preisler
Hi Claudia, Looking at your code you have a cyclic relationship. In other words you are using the same topic for your input and output. If your goal is to always get the most up to date BoxInfo it will not work correctly. The KStream to KTable join matches records from the stream with records

Re: Stream to KTable internals

2021-11-05 Thread Chad Preisler
Chad Preisler wrote: > enforced-processing-total is zero for all missed join occurrences. I > logged all the metrics out at the time my stream processed the missed join, > so let me know if there are any other metics that would help. > > On Wed, Nov 3, 2021 at 9:21 PM Chad Pr

Re: Stream to KTable internals

2021-11-04 Thread Chad Preisler
enforced-processing-total is zero for all missed join occurrences. I logged all the metrics out at the time my stream processed the missed join, so let me know if there are any other metics that would help. On Wed, Nov 3, 2021 at 9:21 PM Chad Preisler wrote: > I'm not sure. When I

Re: Stream to KTable internals

2021-11-03 Thread Chad Preisler
n/#kafka_streams_task_monitoring > > As long as there is input data, we should never do "enforced processing" > and the metric should stay at zero. > > > -Matthias > > On 11/3/21 2:41 PM, Chad Preisler wrote: > > Just a quick update. Setting max.task.idle.ms to 1

Re: Stream to KTable internals

2021-11-03 Thread Chad Preisler
Just a quick update. Setting max.task.idle.ms to 1 (10 seconds) had no effect on this issue. On Tue, Nov 2, 2021 at 6:55 PM Chad Preisler wrote: > No unfortunately it is not the case. The table record is written about 20 > seconds before the stream record. I’ll crank up the time to

Re: Stream to KTable internals

2021-11-02 Thread Chad Preisler
is case? > > > -Matthias > > On 11/2/21 3:13 PM, Chad Preisler wrote: > > Thank you for the information. We are using the Kafka 3.0 client library. > > We are able to reliably reproduce this issue in our test environment > now. I > > removed my timestamp extractor,

Re: Stream to KTable internals

2021-11-02 Thread Chad Preisler
rd was written to the KStream > > topic." So I think Matthias and others have excluded this possibility > while > > trying to help investigate. > > > > If only the matching records from KStream are returned via a single a > > consumer poll call but not the other rec

Re: Stream to KTable internals

2021-10-31 Thread Chad Preisler
You may also check out slides 34-38: > > https://www.confluent.io/kafka-summit-san-francisco-2019/whats-the-time-and-why/ > > There is one corner case: if two records with the same timestamp come > it, it's not defined which one will be processed first. > > Hope this helps. > > >

Re: Stream to KTable internals

2021-10-30 Thread Chad Preisler
what you’re seeing > > > https://www.confluent.io/events/kafka-summit-europe-2021/failing-to-cross-the-streams-lessons-learned-the-hard-way/ > > < > > > https://www.confluent.io/events/kafka-summit-europe-2021/failing-to-cross-the-streams-lessons-learned-the

Re: Stream to KTable internals

2021-10-30 Thread Chad Preisler
’re seeing > https://www.confluent.io/events/kafka-summit-europe-2021/failing-to-cross-the-streams-lessons-learned-the-hard-way/ > < > https://www.confluent.io/events/kafka-summit-europe-2021/failing-to-cross-the-streams-lessons-learned-the-hard-way/ > > > > Gilles > >

Stream to KTable internals

2021-10-29 Thread Chad Preisler
Hello, I have a stream application that does a KStream to KTable left join. We seem to be occasionally missing joins (KTable side is null). I'm wondering if someone can point me to the Kafka streams internal code that reads records for the join? I've poked around the Kafka code base, but there

Re: Amazon MSK Feeback

2020-09-07 Thread Chad Preisler
We use Confluent Cloud. Make sure you consider the opportunity cost of you labor into your MSK cost. Other than creating topics and managing ACLs you won’t have to lift a finger to manage your Confluent Cloud Kafka cluster. That means you get to spend all your time building apps to run your

Re: Failed to rebalance

2019-07-04 Thread Chad Preisler
, Jul 4, 2019 at 9:28 AM Paweł Gontarz wrote: > Thanks Chad, > > Unfortunately that's not our case > > On Thu, Jul 4, 2019 at 4:19 PM Chad Preisler > wrote: > > > I had a similar situation. For us one of our firewall appliances was > > blocking traffic to the bro

Re: Failed to rebalance

2019-07-04 Thread Chad Preisler
I had a similar situation. For us one of our firewall appliances was blocking traffic to the brokers. On Thu, Jul 4, 2019 at 7:43 AM Paweł Gontarz wrote: > Hey all, > > I have seen already in archive an email concerning this, but as a solution > it has been said to upgrade kafka version to 2.1.

kafka test issue

2019-04-13 Thread Chad Preisler
It seems like the Kafka test driver is not compatible with the latest kafka-streams API. When I use the following two dependencies together. org.apache.kafka kafka-streams-test-utils 2.2.0 test org.apache.kafka