Re: Kraft controller readiness checks

2024-04-21 Thread Dima Brodsky
Thanks Luke, this helps for our use case. It does not cover the buildout of a new cluster where there are no brokers, but that should be remedied by kip 919 which looks to be resolved in 3.7.0. ttyl Dima On Sun, Apr 21, 2024 at 9:06 PM Luke Chen wrote: > Hi Frank, > > About your question: >

Re: Kraft controller readiness checks

2024-04-21 Thread Luke Chen
Hi Frank, About your question: > Unless this is already available but not well publicised in the documentation, ideally there should be protocol working on the controller ports that answers to operational questions like “are metadata partitions in sync?”, “has the current controller converged

Re: Failed to initialize processor KSTREAM-AGGREGATE-0000000001

2024-04-21 Thread Matthias J. Sax
Not sure either, but it sounds like a bug to me. Can you reproduce this reliably? What version are you using? It would be best if you could file a Jira ticket and we can take it from there. -Matthias On 4/21/24 5:38 PM, Penumarthi Durga Prasad Chowdary wrote: Hi , I have an issue in

Failed to initialize processor KSTREAM-AGGREGATE-0000000001

2024-04-21 Thread Penumarthi Durga Prasad Chowdary
Hi , I have an issue in kafka-streams while constructing kafka-streams state store windows(TimeWindow and SessionWindow). While kafka-streams processing data sometimes intermittent kafka-streams process throwing below error ThreadName:

SQL Client fails with ClassNotFoundException when Hive dialect is chosen

2024-04-21 Thread Ilya Karpov
Hi, I have *flink 1.19 *installation and start *sql-client locally*, like this: *./bin/sql-client.sh* then execute *SET 'table.sql-dialect' = 'hive'*; and have this exception: *[ERROR] Could not execute SQL statement. Reason:java.lang.ClassNotFoundException: org.antlr.runtime.tree.Tree* I'm