Re: Kafka Tiered Storage Disablement Implement in ZK Mode Question

2024-11-02 Thread Kamal Chandraprakash
ggest you upgrade to KRaft. On Sat, Nov 2, 2024 at 4:15 PM Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Hi Jianfeng, > > As Luke already mentioned, the feature was not tested in ZK mode. > If you still want to enable it in the ZK mode, then: >

Re: Kafka Tiered Storage Disablement Implement in ZK Mode Question

2024-11-02 Thread Kamal Chandraprakash
Hi Jianfeng, As Luke already mentioned, the feature was not tested in ZK mode. If you still want to enable it in the ZK mode, then: 1. Disable the LogConfig#validateNoInvalidRemoteStorageConfigsInZK

Re: Network partition leaves topic-partition leader as sole ISR despite min.isr=2 and producer acks=all settings

2024-08-12 Thread Kamal Chandraprakash
Hi Sabit, Thanks for reporting the issue! This is the last standing replica problem and is being fixed in KIP-966. You can go through the below blog to understand it in detail: https://jack-vanlightly.com/blog/2023/8/17/kafka-kip-966-fixing-the-last-replica-standing-issue#:~:text=Rule%20number%20

Re: How do we usually handle Node disconnected issue for kafka producer

2024-05-04 Thread Kamal Chandraprakash
Hi Sachin, Why do you want to change the default settings? If the connection is open and unused, then it is fair to close the connection after the timeout and reopen it when required. On Fri, May 3, 2024 at 1:06 PM Sachin Mittal wrote: > Hi, > I am using a Kafka producer java client by vert.x f

Re: Kafka followers with higher leader epoch than leader

2024-03-31 Thread Kamal Chandraprakash
Hi, The follower is not able to sync-up with the leader due to epochs diverged between leader and follower. To confirm this, you can enable request logger and check the diverging-epoch field in the fetch-response: https://sourcegraph.com/github.com/apache/kafka@a640a81040f6ef6f85819b60194f0394f5f

Re: Script to delete list of topics

2024-01-23 Thread Kamal Chandraprakash
for topic in `cat /tmp/topics.txt`; do echo $topic; sh kafka-topics.sh --bootstrap-servers localhost:9092 --topic $topic --delete; done # you can also delete the topics by wildcard sh kafka-topics.sh --bootstrap-servers localhost:9092 --topic abc.* --delete On Tue, Jan 23, 2024 at 11:50 AM sunil c

Re: [VOTE] 3.6.0 RC2

2023-10-02 Thread Kamal Chandraprakash
+1 (non-binding) 1. Built the source from 3.6 branch in scala 2.12 and 2.13 2. Ran all the unit and integration tests. 3. Ran quickstart and verified the produce-consume on a 3 node cluster. 4. Verified the tiered storage functionality with local-tiered storage. Thanks, Kamal On Mon, Oct 2, 2023

Re: Release plan required for version 3.5.1

2023-07-25 Thread Kamal Chandraprakash
Hi Sahil, Apache Kafka 3.5.1 is already released: https://kafka.apache.org/downloads On Wed, Jul 26, 2023 at 9:08 AM Sahil Sharma D wrote: > Gentle reminder-2 > > -Original Message- > From: Sahil Sharma D > Sent: 12 July 2023 09:51 AM > To: users@kafka.apache.org > Subject: RE: Release

Re: Offset commit consistently timing out in transaction

2021-01-08 Thread Kamal Chandraprakash
Timeout in offset commit request has been fixed recently. This issue seems to be more of KAFKA-8334 . On Mon, Jan 4, 2021 at 3:53 PM Kindernay Oliver wrote: > Hello, > > we are experiencing problems with offset commit timing out on brokers. > Th

Re: Max poll interval and timeouts

2020-03-25 Thread Kamal Chandraprakash
With group coordination protocol, you only have to increase the ` max.poll.interval.ms` / `max.poll.records`. Ignore the above messages. Consumer heartbeats are processed in a separate thread. On Wed, Mar 25, 2020 at 2:35 PM Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote:

Re: Max poll interval and timeouts

2020-03-25 Thread Kamal Chandraprakash
24, 2020 at 11:56 PM Ryan Schachte wrote: > Don't I lose consumer group coordination with assign? > > On Mon, Mar 23, 2020 at 11:49 PM Kamal Chandraprakash < > kamal.chandraprak...@gmail.com> wrote: > > > Hi Ryan, > > > > The maxPollInterval waits fo

Re: MirrorMaker2 - uneven loadbalancing

2020-03-24 Thread Kamal Chandraprakash
Hi Peter, Not sure this is what you're looking for - https://issues.apache.org/jira/browse/KAFKA-9352 On Mon, Mar 23, 2020 at 11:37 PM Ryanne Dolan wrote: > Thanks Peter for running this experiment. That looks sorta normal. It looks > like Connect is deciding to use 10 total tasks and doesn't c

Re: Max poll interval and timeouts

2020-03-23 Thread Kamal Chandraprakash
Hi Ryan, The maxPollInterval waits for at-most the given time duration and returns ASAP even if a single record is available. If you want to collect data once 30-45 minutes, better to use the Consumer with `assign` mode and poll for records once in 30 minutes. If you're using the consumer with `

Re: [ANNOUNCE] New committer: John Roesler

2019-11-12 Thread Kamal Chandraprakash
Congrats John! On Wed, Nov 13, 2019 at 7:57 AM Dong Lin wrote: > Congratulations John! > > On Tue, Nov 12, 2019 at 1:56 PM Guozhang Wang wrote: > > > Hi Everyone, > > > > The PMC of Apache Kafka is pleased to announce a new Kafka committer, > John > > Roesler. > > > > John has been contributing

Re: list of pattern processed topic list

2019-08-19 Thread Kamal Chandraprakash
You can use the KafkaConsumer#assignment() method to get all the assigned topic-partitions for that consumer instance. But, you've to periodically call poll method to get the latest assignment which may return records. This shortcoming is actively discussed in the below threads. https://mail-archi

Re: [VOTE] 2.3.0 RC3

2019-06-21 Thread Kamal Chandraprakash
+1 (non-binding) * Ran unit and integration test on 2.11 and 2.12 * Verified quick start * Ran internal apps on the 3 node cluster On Thu, Jun 20, 2019 at 3:33 AM Colin McCabe wrote: > Hi all, > > We discovered some problems with the second release candidate (RC2) of > 2.3.0. Specifically, KAF

Re: Kafka delaying message

2019-05-26 Thread Kamal Chandraprakash
If you have a dedicated topicPartition for delayed messages, you can pause that partition for 15 min to avoid blocking the polling thread. On Thu, May 2

Re: KeeperException

2019-05-14 Thread Kamal Chandraprakash
Those are not errors. See the logs are logged in INFO mode. https://stackoverflow.com/a/48067058/3209010 On Wed, May 15, 2019 at 10:44 AM Gagan Sabharwal wrote: > Hi team, > > Any pointers on the same ? > > Regards > Gagan > > On Mon, May 13, 2019 at 11:24 AM Gagan Sabharwal > wrote: > > > Hi

Re: Kafka transaction between 2 kafka clusters

2019-05-10 Thread Kamal Chandraprakash
MirrorMaker 2.0 stores the offsets of one cluster in another. So, you can read the offsets from the same cluster once this KIP is implemented. https://cwiki.apache.org/confluence/display/KAFKA/KIP-382%3A+MirrorMaker+2.0#KIP-382:MirrorMaker2.0-RemoteClusterUtils On Fri, May 10, 2019 at 12:29 PM Em

Re: Kafka upgrade process details

2019-05-10 Thread Kamal Chandraprakash
Hi, In Kafka v2.1.0, the OffsetCommit Request/Response schema version is changed to v4 for the *__consumer_offsets* topic. If you upgrade Kafka to v2.1.0 & higher and changed the inter.broker.protocol version to 2.1, then you cannot revert back to older versions as it doesn't know how to read the

Re: Best Practice Scaling Consumers

2019-05-06 Thread Kamal Chandraprakash
1. Yes, you may have to overprovision the number of partitions to handle the load peaks. Refer this document to choose the no. of partitions. 2. KIP-429

Re: Restart process after adding control.plane.listener.name config in Kafka 2.2.0

2019-05-06 Thread Kamal Chandraprakash
Migration plan for this change is mentioned in the KIP-291 . Take a look at the migration plan. Migration plan: 2 rounds of rolling upgrades are needed to pick up the >

Re: measuring incoming bytes

2019-02-02 Thread Kamal Chandraprakash
In Yammer metrics, the 15MinuteRate provided by the Meter is not a pure average for the last 15 minutes. It's a Exponential Weighted Moving Average. The value is calculated by taking 36% of weighted average from the broker start time and 64 % of weighted average from the last 15 minutes. I initial

Re: Very long consumer rebalances

2018-08-09 Thread Kamal Chandraprakash
In v0.10.0.1, consumer heartbeat background thread feature is not available. Lot of users faced similar errors. So, KIP-62 is proposed. You have to update your Kafka version

Re: How set properly infinite retention

2018-07-30 Thread Kamal Chandraprakash
log.retention.ms = 9223372036854775807 (Long.MAX_VALUE) On Mon, Jul 30, 2018 at 3:04 PM David Espinosa wrote: > Hi thanks a lot for the reply. > > The thing is that I need compaction to delete some messages (for GDPR > purposes), and for that I need the log cleaner to be enabled (with > policy=

Re: Facing Duplication in consumer

2018-05-28 Thread Kamal Chandraprakash
If the consumer is idle (not doing any commit) for more than a day, the offsets will be wiped out. Rf. offsets.retention.minutes property in Kafka documentation. On Tue, May 29, 2018 at 9:49 AM, Shantanu Deshmukh wrote: > Which Kafka version? > > On Mon, May 28, 2018 at 9:09 PM Dinesh Subramani

Re: 答复: [ANNOUNCE] New Committer: Dong Lin

2018-03-28 Thread Kamal Chandraprakash
Congratulations, Dong! On Thu, Mar 29, 2018 at 7:14 AM, Manikumar wrote: > Congrats, Dong! > > On Thu, Mar 29, 2018 at 6:45 AM, Tao Feng wrote: > > > Congrats Dong! > > > > On Wed, Mar 28, 2018 at 5:15 PM Dong Lin wrote: > > > > > Thanks everyone!! > > > > > > It is my great pleasure to be par

Re: kafka user

2017-11-01 Thread Kamal Chandraprakash
Please follow the instructions listed here https://kafka.apache.org/contact You have to send a mail to 'users-subscr...@kafka.apache.org' to subscribe to user list. On Tue, Oct 31, 2017 at 1:06 PM, Karthigeyan wrote: > Hi , > > pls add to the user group. > > Thanks , > > Karthigeyan > >

KTable-KTable Join Semantics on NULL Key

2017-09-08 Thread Kamal Chandraprakash
Hi Kafka Users, KTable-KTable Join Semantics is explained in detailed [here][1]. But, it's not clear when the input record is , some times the output records are generated and in some cases it's not. It will be helpful, if someone explain on how the output records are generated for all the 3

Re: Reduce Kafka Client logging

2017-09-08 Thread Kamal Chandraprakash
add this lines at the end of your log4j.properties, log4j.logger.org.apache.kafka.clients.producer=WARN On Thu, Sep 7, 2017 at 5:27 PM, Raghav wrote: > Hi Viktor > > Can you pleas share the log4j config snippet that I should use. My Java > code's current log4j looks like this. How should I add

Re: Why does kafka-consumer-groups show the topics written to too in its describe

2017-08-21 Thread Kamal Chandraprakash
`through` = `to` + `stream` operation. So, the consumer-groups command showing the "fname-stream" topic. Use `to`, if you just want to write the output to the topic. -- Kamal On Mon, Aug 21, 2017 at 12:05 PM, Sachin Mittal wrote: > Folks any thoughts on this. > Basically I want to know on what