Re: [DISCUSS] KIP-1021: Allow to get last stable offset (LSO) in kafka-get-offsets.sh

2024-09-21 Thread Kamal Chandraprakash
hen wrote: > Hi Ahmed, > > Are you still interested in this KIP? > Should we take it over if you're busy on other things? > > Thanks. > Luke > > > On Wed, Mar 27, 2024 at 12:26 PM Kamal Chandraprakash < > kamal.chandraprak...@gmail.com> wrote: > >

Re: [VOTE] KIP-1058: Txn consumer exerts pressure on remote storage when reading non-txn topic

2024-09-15 Thread Kamal Chandraprakash
Bumping this thread for vote. PTAL. On Mon, Sep 9, 2024 at 2:01 PM Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Hi all, > > I'd like to open voting for KIP-1058. This KIP improves the consumer > reading from remote storage when READ_COMMITTED isolation l

[jira] [Created] (KAFKA-17552) Handle LIST_OFFSETS request for max_timestamp when remote storage is enabled

2024-09-13 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-17552: Summary: Handle LIST_OFFSETS request for max_timestamp when remote storage is enabled Key: KAFKA-17552 URL: https://issues.apache.org/jira/browse/KAFKA-17552

Re: [DISCUSS] Single broker failures causing offline partitions

2024-09-11 Thread Kamal Chandraprakash
Hi Haruki, We are also interested in this issue. > The problem is how to identify such "eligible" replicas. Is the leader election automated to find the replica with the highest offset and latest epoch? If yes, could you please open a PR for it? When a broker goes down, it might be serving lead

[VOTE] KIP-1058: Txn consumer exerts pressure on remote storage when reading non-txn topic

2024-09-09 Thread Kamal Chandraprakash
Hi all, I'd like to open voting for KIP-1058. This KIP improves the consumer reading from remote storage when READ_COMMITTED isolation level is enabled. PTAL. KIP-1058

Re: [ANNOUNCE] New committer: Jeff Kim

2024-09-09 Thread Kamal Chandraprakash
Congrats, Jeff! On Mon, Sep 9, 2024 at 12:20 PM Chia-Ping Tsai wrote: > Congratulations, Jeff! thanks for you to bring the great new group > coordinator! > > Best, > Chia-Ping > > David Jacot 於 2024年9月9日 週一 下午2:44寫道: > > > Hi all, > > > > The PMC of Apache Kafka is pleased to announce a new Ka

Re: [VOTE] KIP-1075: Introduce delayed remote list offsets purgatory to make LIST_OFFSETS async

2024-08-29 Thread Kamal Chandraprakash
Hi all, The vote is closed. The KIP has been accepted with 4 +1 binding votes (Satish, Luke, Christo, and Chia-Ping). Thank you all for the participation! -- Kamal On Thu, Aug 29, 2024 at 5:14 PM Chia-Ping Tsai wrote: > +1 (binding) > > On 2024/08/19 05:09:27 Kamal Chandraprak

Re: [DISCUSS] KIP-1075: Introduce delayed remote list offsets purgatory to make LIST_OFFSETS async

2024-08-29 Thread Kamal Chandraprakash
Hi Chia-Ping, Thanks for the review! This is a good point to add provision to convey the timeout from client to the server. Updated the KIP to add a new 'TimeoutMs' field in the ListOffsetsRequest and bumped the version to 10. To retain the existing ListOffsets request behaviour: 1. AdminClient

Re: [ANNOUNCE] New committer: Lianet Magrans

2024-08-28 Thread Kamal Chandraprakash
Congrats Lianet! On Thu, Aug 29, 2024 at 8:07 AM Luke Chen wrote: > Congratulations Lianet! > > Luke > > On Thu, Aug 29, 2024 at 9:54 AM Lianet M. wrote: > > > Thank you very much everyone! It truly takes the great shared knowledge > you > > all put out there with amazing reviews and discussion

Re: [DISCUSS] KIP-1075: Introduce delayed remote list offsets purgatory to make LIST_OFFSETS async

2024-08-26 Thread Kamal Chandraprakash
Hi all, This KIP needs one more binding vote to pass. Please review the KIP. Thanks! -- Kamal On Sat, Aug 24, 2024 at 8:11 AM Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Hi Jun, > > Thanks for the review! > > 1. purgatoryName = RemoteListOffsets >

Re: [DISCUSS] KIP-1075: Introduce delayed remote list offsets purgatory to make LIST_OFFSETS async

2024-08-23 Thread Kamal Chandraprakash
al level like the fetch > request? > > Jun > > On Fri, Aug 2, 2024 at 12:04 AM Kamal Chandraprakash < > kamal.chandraprak...@gmail.com> wrote: > > > Hi all, > > > > I would like to start a discussion thread on KIP-1075 > > < > >

[VOTE] KIP-1075: Introduce delayed remote list offsets purgatory to make LIST_OFFSETS async

2024-08-18 Thread Kamal Chandraprakash
Hi, I would like to open voting for KIP-1075. I've addressed the review comments in the discussion thread. Please vote if the proposal looks good. https://cwiki.apache.org/confluence/plugins/servlet/mobile?contentId=315494171#content/view/315494171 Thanks, Kamal

Re: [DISCUSS] KIP-1075: Introduce delayed remote list offsets purgatory to make LIST_OFFSETS async

2024-08-17 Thread Kamal Chandraprakash
Hi all, If there are no more comments, I'll start a vote soon. Thanks, Kamal On Wed, Aug 14, 2024 at 5:05 PM Luke Chen wrote: > Hi Kamal, > > Thanks for the update. > LGTM. > > Luke > > On Wed, Aug 14, 2024 at 7:25 PM Kamal Chandraprakash < > kamal.chandr

Re: [DISCUSS] KIP-1075: Introduce delayed remote list offsets purgatory to make LIST_OFFSETS async

2024-08-14 Thread Kamal Chandraprakash
Hi all, > I saw we added some new configs/metrics. I have removed the recent changes to the public interfaces to limit the scope of the KIP to minimum. PTAL. Thanks, Kamal On Wed, Aug 14, 2024 at 9:58 AM Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Hi Luk

Re: [DISCUSS] KIP-1075: Introduce delayed remote list offsets purgatory to make LIST_OFFSETS async

2024-08-13 Thread Kamal Chandraprakash
we be consistent to use > `reader`, instead of `read`? > > LC7: We should set a default value for the newly introduced configs and > written in KIP. > > Thanks. > Luke > > On Tue, Aug 13, 2024 at 8:47 PM Kamal Chandraprakash < > kamal.chandraprak...@gmail.com> wro

Re: [DISCUSS] KIP-1058: Txn consumer exerts pressure on remote storage when reading non-txn topic

2024-08-13 Thread Kamal Chandraprakash
Bumping this thread for KIP review! We can go for the simplest solution that is proposed in this KIP and it can be improved in the subsequent iteration. PTAL. Thanks, Kamal On Fri, Aug 2, 2024 at 11:42 AM Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Hi Divij, >

Re: [DISCUSS] KIP-1075: Introduce delayed remote list offsets purgatory to make LIST_OFFSETS async

2024-08-13 Thread Kamal Chandraprakash
receives the "response" from > server? > > LC4: I found the different consumer and admin behavior when setting > "request.timeout" and "default.api.timeout" is confusing. Are they expected > or a bug? > > Thank you. > Luke > > > On Thu, Aug 8

Re: [DISCUSS] KIP-1075: Introduce delayed remote list offsets purgatory to make LIST_OFFSETS async

2024-08-08 Thread Kamal Chandraprakash
Hi Luke, Thanks for the review! LC1: When the consumer starts to read data, then it might need the below offsets: earliest, latest, and last-committed-offset based on the "auto.offset.reset" config. The earliest and latest offsets have special timestamps -2 and -1, those timestamp corresponding

Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-08-07 Thread Kamal Chandraprakash
which > > means `retention.ms` value to be used) > > > > So, basically, we don't change the original design, just want to make > sure > > users are aware of the retention policy change after disabling remote log > > copy. > > > > Let me know if you have

Re: [DISCUSS] KIP-1075: Introduce delayed remote list offsets purgatory to make LIST_OFFSETS async

2024-08-05 Thread Kamal Chandraprakash
Bumping this thread. Please take a look. On Fri, Aug 2, 2024 at 12:32 PM Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Hi all, > > I would like to start a discussion thread on KIP-1075 > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-1075%3A+Int

[DISCUSS] KIP-1075: Introduce delayed remote list offsets purgatory to make LIST_OFFSETS async

2024-08-02 Thread Kamal Chandraprakash
Hi all, I would like to start a discussion thread on KIP-1075 to make the remote LIST_OFFSETS an async operation. The KIP is here: https://cwiki.apache.org/co

Re: [DISCUSS] KIP-1058: Txn consumer exerts pressure on remote storage when reading non-txn topic

2024-08-01 Thread Kamal Chandraprakash
we would not have to > iterate through the metadata for the dominant case where the partition had > no transactions at all. > > -- > Divij Vaidya > > > > On Tue, Jun 25, 2024 at 11:42 AM Kamal Chandraprakash < > kamal.chandraprak...@gmail.com> wrote: > &g

Re: [kafka-clients] [ANNOUNCE] Apache Kafka 3.8.0

2024-07-30 Thread Kamal Chandraprakash
wxy, Dung Ha, Edoardo Comar, Eduwer > >> Camacaro, Emanuele Sabellico, Erik van Oosten, Eugene Mitskevich, Fan > >> Yang, Federico Valeri, Fiore Mario Vitale, flashmouse, Florin Akermann, > >> Frederik Rouleau, Gantigmaa Selenge, Gaurav Narula, ghostspiders, > >>

Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-07-26 Thread Kamal Chandraprakash
et `remote.copy.disabled= > true`. If you want to disable remote storage and delete all remote data, > please set `remote.storage.enable=false,remote.log.delete.on.disable=true`. > > I've updated the KIP. Please take a look when available. > > https://cwiki.apache.org/confl

Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-07-25 Thread Kamal Chandraprakash
Correction: (2): Wait for all the remote segments to be deleted async due to breach by retention time (or) size, then set the `remote.storage.enable = false` and `remote.log.delete.on.disable = true`. This step is optional. On Thu, Jul 25, 2024 at 11:13 PM Kamal Chandraprakash

Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-07-25 Thread Kamal Chandraprakash
Hi Chia-Ping, Thanks for the review! > If so, what is the purpose of `remote.log.delete.on.disable=false`? IIUC, the purpose of `remote.log.delete.on.disable` is to get explicit confirmation from the user before deleting the remote log segments. The concern raised in the thread is that if the u

Re: [DISCUSS] KIP-1051 Statically configured log replication throttling

2024-07-18 Thread Kamal Chandraprakash
about having it as a gauge for number of partitions that are > currently leader of follower throttled (similar to the URP metric)? > > Kind regards, > Harry > > On Thursday, 11 July 2024 at 19:02, Kamal Chandraprakash < > kamal.chandraprak...@gmail.co

Re: [DISCUSS] KIP-1066: Mechanism to cordon brokers and log directories

2024-07-14 Thread Kamal Chandraprakash
rectories cordoned, and > > you still need to describe a specific broker's configuration to find > > the "name" of a log directory you want to cordon. > > > > I think an easy way to get an overall view of the cordoned log > > directories/brokers will

Re: [DISCUSS] KIP-1066: Mechanism to cordon brokers and log directories

2024-07-11 Thread Kamal Chandraprakash
Hi Mickael, Thanks for the KIP! This is a useful feature which helps to decommission the nodes by essentially creating a new replica exclude broker list. To cordon a list of brokers, we have to apply the config on each of the broker nodes and similarly to see the list of cordoned brokers, we hav

Re: [DISCUSS] KIP-1051 Statically configured log replication throttling

2024-07-11 Thread Kamal Chandraprakash
Hi Harry Fallows, Thanks for the KIP! I went over both the KIP-1051 and KIP-1009. Assuming that the leader.replication.throttled.replicas and follower.replication.throttled.replicas are set to Wildcard (*) to apply for all the partitions in the broker. If we set a static value for leader and foll

Re: [DISCUSS] KIP-1058: Txn consumer exerts pressure on remote storage when reading non-txn topic

2024-06-25 Thread Kamal Chandraprakash
Bump. Please review this proposal. On Mon, Jun 17, 2024 at 6:55 PM Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Divij, > > Thanks for the review! Updated the KIP with 1, 2, 3, and 4 review > comments. > > > 4. Potential alternative - Instead of h

Re: [VOTE] KIP-1057: Add remote log metadata flag to the dump log tool

2024-06-21 Thread Kamal Chandraprakash
Hi Federico, Thanks for the KIP! +1 from me. On Fri, Jun 21, 2024 at 5:47 PM Luke Chen wrote: > Hi Fede, > > Thanks for the KIP! > +1 from me. > > Luke > > On Fri, Jun 21, 2024 at 6:44 PM Federico Valeri > wrote: > > > Hi all, I'd like to kick off a vote on KIP-1057. > > > > Design doc: > > >

Re: [DISCUSS] KIP-1057: Add remote log metadata flag to the dump log tool

2024-06-20 Thread Kamal Chandraprakash
e dump-logs tool with custom decoder > > - Running kafka-consumer.sh on the topic. > > > > ~Satish. > > > > > > ~Satish. > > > > > > > > On Mon, 17 Jun 2024 at 15:55, Federico Valeri > > wrote: > > > > > > Hi Kamal,

Re: [DISCUSS] KIP-1058: Txn consumer exerts pressure on remote storage when reading non-txn topic

2024-06-17 Thread Kamal Chandraprakash
arameter instead of > "topicIdPartition"? Suggesting because isTxnIdEmpty is not a property of a > partition, instead it's a property of a specific segment. > > Looking forward to hearing your thoughts about the alternatives. Let's get > this fixed. > > -- &g

Re: [DISCUSS] KIP-1057: Add remote log metadata flag to the dump log tool

2024-06-17 Thread Kamal Chandraprakash
We can use the console-consumer to read the contents of the `__remote_log_metadata` topic. Why are we proposing a new tool? sh kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic __remote_log_metadata --consumer-property exclude.internal.topics=false --formatter org.apache.kafka.s

[DISCUSS] KIP-1058: Txn consumer exerts pressure on remote storage when reading non-txn topic

2024-06-17 Thread Kamal Chandraprakash
Hi all, I have opened a KIP-1058 to reduce the pressure on remote storage when transactional consumers are reading non-txn topics from remote storage. https://

[jira] [Reopened] (KAFKA-15420) Kafka Tiered Storage V1

2024-06-17 Thread Kamal Chandraprakash (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15420?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kamal Chandraprakash reopened KAFKA-15420: -- > Kafka Tiered Storage V1 > --- > >

[jira] [Resolved] (KAFKA-15420) Kafka Tiered Storage V1

2024-06-17 Thread Kamal Chandraprakash (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15420?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kamal Chandraprakash resolved KAFKA-15420. -- Resolution: Fixed > Kafka Tiered Storage

[jira] [Created] (KAFKA-16948) Reset tier lag metrics on becoming follower

2024-06-12 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-16948: Summary: Reset tier lag metrics on becoming follower Key: KAFKA-16948 URL: https://issues.apache.org/jira/browse/KAFKA-16948 Project: Kafka

Re: [DISCUSS] RemoteLogSegmentMetadataRecord Schema Evolution

2024-06-12 Thread Kamal Chandraprakash
Ignore the above message. Went over the Kafka message definition README <https://github.com/apache/kafka/blob/trunk/clients/src/main/resources/common/message/README.md> guide and learnt about the tagged fields. On Tue, Jun 11, 2024 at 10:32 PM Kamal Chandraprakash < kamal.chandraprak...@

[DISCUSS] RemoteLogSegmentMetadataRecord Schema Evolution

2024-06-11 Thread Kamal Chandraprakash
Hi, We have defined the remote log segment metadata schemas with type as "metadata" (controller records) in #10271 : 1. RemoteLogSegmentMetadataRecord.json

[jira] [Resolved] (KAFKA-15777) Configurable remote fetch bytes per partition from Consumer

2024-06-09 Thread Kamal Chandraprakash (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15777?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kamal Chandraprakash resolved KAFKA-15777. -- Resolution: Won't Fix > Configurable remote fetch bytes per partit

[jira] [Created] (KAFKA-16904) Metric to measure the latency of remote read requests

2024-06-06 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-16904: Summary: Metric to measure the latency of remote read requests Key: KAFKA-16904 URL: https://issues.apache.org/jira/browse/KAFKA-16904 Project: Kafka

[jira] [Created] (KAFKA-16882) Migrate RemoteLogSegmentLifecycleTest to new test infra

2024-06-03 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-16882: Summary: Migrate RemoteLogSegmentLifecycleTest to new test infra Key: KAFKA-16882 URL: https://issues.apache.org/jira/browse/KAFKA-16882 Project

[jira] [Created] (KAFKA-16780) Txn consumer exerts pressure on remote storage when reading non-txn topic

2024-05-16 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-16780: Summary: Txn consumer exerts pressure on remote storage when reading non-txn topic Key: KAFKA-16780 URL: https://issues.apache.org/jira/browse/KAFKA-16780

Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-05-15 Thread Kamal Chandraprakash
Hi Christo, Thanks for the KIP and incorporating the review comments. Please update the KIP with the latest details. +1 (non-binding). Thanks, Kamal On Wed, May 15, 2024 at 3:18 PM Luke Chen wrote: > Hi Christo, > > In addition to the minor comments left in the discussion thread, it LGTM. >

Re: [VOTE] KIP-1018: Introduce max remote fetch timeout config

2024-05-11 Thread Kamal Chandraprakash
; wrote: > > > > > +1 non binding > > > > > > Thanks > > > > > > On Thu, May 9, 2024 at 12:05 PM Luke Chen wrote: > > > > > > > > Hi Kamal, > > > > > > > > Thanks for the KIP! > >

[jira] [Created] (KAFKA-16696) Remove the in-memory implementation of RSM and RLMM

2024-05-09 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-16696: Summary: Remove the in-memory implementation of RSM and RLMM Key: KAFKA-16696 URL: https://issues.apache.org/jira/browse/KAFKA-16696 Project: Kafka

Re: [DISCUSS] KIP-950: Tiered Storage Disablement

2024-05-09 Thread Kamal Chandraprakash
Deletion when the policy is retain happens asynchronously due to > expiration. I think that deletion when the policy is delete ought to (at > least for the initial implementation) happen synchronously. Should people > run into timeout problems we can always then have a follow-up KIP wher

Re: [DISCUSS] KIP-950: Tiered Storage Disablement

2024-05-07 Thread Kamal Chandraprakash
ents to delete is huge. We can further extend the same approach for the topic deletion requests. Also, Could you please update the state diagram about the transitions? It is not clear when to transit from DISABLING to DISABLED state? -- Kamal On Mon, May 6, 2024 at 6:55 PM Kamal Chandraprakash &

[VOTE] KIP-1018: Introduce max remote fetch timeout config

2024-05-06 Thread Kamal Chandraprakash
Hi all, We would like to start a voting thread for KIP-1018: Introduce max remote fetch timeout config for DelayedRemoteFetch requests. The KIP is available on https://cwiki.apache.org/confluence/display/KAFKA/KIP-1018%3A+Introduce+max+remote+fetch+timeout+config+for+DelayedRemoteFetch+requests

Re: [DISCUSS] KIP-1018: Introduce max remote fetch timeout config

2024-05-06 Thread Kamal Chandraprakash
Hi all, If there are no more comments, I'll open a VOTE thread. -- Kamal On Sat, May 4, 2024 at 8:39 AM Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Luke, > > Thanks for the review! > > DelayedFetch and DelayedRemoteFetch are orthogonal calls &g

Re: [DISCUSS] KIP-1018: Introduce max remote fetch timeout config

2024-05-03 Thread Kamal Chandraprakash
ere, is it > > possible > > > to add 2 metrics, one at the RequestMetrics level (even if it is > > > total-time-ms - (all other times)) and one at what you are proposing? > As > > an > > > operator I would find it strange to not see the metric in the > > &g

Re: [DISCUSS] KIP-1018: Introduce max remote fetch timeout config

2024-05-02 Thread Kamal Chandraprakash
to not see the metric in the > RequestMetrics. > > Your thoughts? > > Best, > Christo > > On Sun, 28 Apr 2024 at 10:52, Kamal Chandraprakash < > kamal.chandraprak...@gmail.com> wrote: > > > Christo, > > > > Updated the KIP with the remote fetch lat

Re: [DISCUSS] KIP-1018: Introduce max remote fetch timeout config

2024-04-28 Thread Kamal Chandraprakash
Christo, Updated the KIP with the remote fetch latency metric. Please take another look! -- Kamal On Sun, Apr 28, 2024 at 12:23 PM Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Hi Federico, > > Thanks for the suggestion! Updated the config name to " > r

Re: [DISCUSS] KIP-1018: Introduce max remote fetch timeout config

2024-04-27 Thread Kamal Chandraprakash
Sat, Apr 27, 2024 at 4:03 PM Federico Valeri wrote: > Hi Kamal, it looks like all TS configurations starts with "remote." > prefix, so I was wondering if we should name it > "remote.fetch.max.wait.ms". > > On Fri, Apr 26, 2024 at 7:07 PM Kamal Chandraprakash

Re: [DISCUSS] KIP-1018: Introduce max remote fetch timeout config

2024-04-26 Thread Kamal Chandraprakash
Hi all, If there are no more comments, I'll start a vote thread by tomorrow. Please review the KIP. Thanks, Kamal On Sat, Mar 30, 2024 at 11:08 PM Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Hi all, > > Bumping the thread. Please review this KIP. Thank

Re: [VOTE] KIP-1023: Follower fetch from tiered offset

2024-04-25 Thread Kamal Chandraprakash
+1 (non-binding). Thanks for the KIP! -- Kamal On Fri, Apr 26, 2024 at 11:00 AM Abhijeet Kumar wrote: > Hi All, > > I would like to start the vote for KIP-1023 - Follower fetch from tiered > offset > > The KIP is here: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1023%3A+Follower+f

Re: [ANNOUNCE] New committer: Igor Soarez

2024-04-24 Thread Kamal Chandraprakash
Congratulations, Igor! On Thu, Apr 25, 2024, 06:11 Luke Chen wrote: > Congrats, Igor! > > On Thu, Apr 25, 2024 at 6:10 AM Matthias J. Sax wrote: > > > Congrats! > > > > On 4/24/24 2:29 PM, Bill Bejeck wrote: > > > Congrats Igor! > > > > > > -Bill > > > > > > On Wed, Apr 24, 2024 at 2:37 PM Tom

[jira] [Created] (KAFKA-16605) Fix the flaky LogCleanerParameterizedIntegrationTest

2024-04-23 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-16605: Summary: Fix the flaky LogCleanerParameterizedIntegrationTest Key: KAFKA-16605 URL: https://issues.apache.org/jira/browse/KAFKA-16605 Project: Kafka

Re: [ANNOUNCE] New Kafka PMC Member: Greg Harris

2024-04-13 Thread Kamal Chandraprakash
Congratulations, Greg! On Sun, Apr 14, 2024 at 8:57 AM Yash Mayya wrote: > Congrats Greg! > > On Sun, 14 Apr, 2024, 05:56 Randall Hauch, wrote: > > > Congratulations, Greg! > > > > On Sat, Apr 13, 2024 at 6:36 PM Luke Chen wrote: > > > > > Congrats, Greg! > > > > > > On Sun, Apr 14, 2024 at 7:

Re: [DISCUSS] KIP-1026: Handling producer snapshot when upgrading from < v2.8.0 for Tiered Storage

2024-04-11 Thread Kamal Chandraprakash
logs? > > Thanks. > Luke > > > On Wed, Apr 3, 2024 at 11:59 PM Arpit Goyal > wrote: > >> Thanks @Kamal Chandraprakash Greg >> Harris >> I currently do not have detailed understanding on the behaviour when empty >> producer snapshot restored. I will

[jira] [Resolved] (KAFKA-16456) Can't stop kafka debug logs

2024-04-09 Thread Kamal Chandraprakash (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16456?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kamal Chandraprakash resolved KAFKA-16456. -- Resolution: Not A Problem You can also dynamically change the broker loggers

[jira] [Created] (KAFKA-16454) Snapshot the state of remote log metadata for all the partitions

2024-04-01 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-16454: Summary: Snapshot the state of remote log metadata for all the partitions Key: KAFKA-16454 URL: https://issues.apache.org/jira/browse/KAFKA-16454

Re: [DISCUSS] KIP-1007: Introduce Remote Storage Not Ready Exception

2024-03-31 Thread Kamal Chandraprakash
t; exception is able to understand what it means. We can discuss that during > the PR review though. > > The KIP looks good to me. > > -- > Divij Vaidya > > > > On Fri, Jan 5, 2024 at 10:44 AM Satish Duggana > wrote: > > > Thanks for the KIP Kamal, LGTM. &

Re: [VOTE] KIP-1007: Introduce Remote Storage Not Ready Exception

2024-03-31 Thread Kamal Chandraprakash
n/java/org/apache/kafka/clients/consumer/internals/FetchCollector.java?L325 On Sun, Jan 7, 2024 at 5:36 PM Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Thank you for all the votes! I'm closing the vote thread as it is open for > more than 72 hours. > The

[jira] [Created] (KAFKA-16452) Bound highwatermark offset to range b/w local-log-start-offset and log-end-offset

2024-03-31 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-16452: Summary: Bound highwatermark offset to range b/w local-log-start-offset and log-end-offset Key: KAFKA-16452 URL: https://issues.apache.org/jira/browse/KAFKA-16452

Re: [DISCUSS] KIP-1018: Introduce max remote fetch timeout config

2024-03-30 Thread Kamal Chandraprakash
Hi all, Bumping the thread. Please review this KIP. Thanks! On Thu, Feb 1, 2024 at 9:11 PM Kamal Chandraprakash < kamal.chandraprak...@gmail.com> wrote: > Hi Jorge, > > Thanks for the review! Added your suggestions to the KIP. PTAL. > > The `fetch.max.wait.ms` config will b

Re: [DISCUSS] KIP-1021: Allow to get last stable offset (LSO) in kafka-get-offsets.sh

2024-03-26 Thread Kamal Chandraprakash
Hi Ahmed, Thanks for the KIP! It will be a useful addition to know the last-stable-offset (LSO) for a partition. Motivation section is not clear to me. Can you document the scenarios on how the exposed LSO can be used for txn support/debug? -- Kamal On Thu, Mar 21, 2024 at 11:30 PM Justine Olsha

Re: [DISCUSS] KIP-950: Tiered Storage Disablement

2024-03-26 Thread Kamal Chandraprakash
Hi, Thanks for the KIP! Overall the KIP looks good and covered most of the items. 1. Could you explain how the brokers will handle the DisableRemoteTopic API request? 2. Who will initiate the controller interaction sequence? Does the controller listens for topic config updates and initiate the d

Re: [ANNOUNCE] New committer: Christo Lolov

2024-03-26 Thread Kamal Chandraprakash
Congrats Christo! On Tue, Mar 26, 2024, 17:42 Bruno Cadonna wrote: > Congrats, Christo! > > Well deserved! > > Best, > Bruno > > On 3/26/24 1:04 PM, Luke Chen wrote: > > Hi, Everyone, > > > > The PMC of Apache Kafka is pleased to announce a new Kafka committer: > > Christo Lolov. > > > > Christo

Re: [DISCUSS] KIP-1026: Handling producer snapshot when upgrading from < v2.8.0 for Tiered Storage

2024-03-26 Thread Kamal Chandraprakash
required for topics migrated from < 2.8 > > version. This will not break the existing contract with the plugin. Yes, > > metrics do not make sense here as of now. > > Greg, @Kamal Chandraprakash WDYT ? > > Arpit Goyal > > 8861094754 > > > > > > On S

Re: [DISCUSS] KIP-1026: Handling producer snapshot when upgrading from < v2.8.0 for Tiered Storage

2024-03-15 Thread Kamal Chandraprakash
Hi Arpit, Thanks for the KIP! There is an open ticket [1] to generate the empty producer snapshot for segments which lacks one. Tiered storage is supported from IBP 2.8-IV1 which mandates that all the segments should have the producer snapshots. If we make the producer snapshot optional, then we

Re: [DISCUSS] Minimum constraint for segment.ms

2024-03-13 Thread Kamal Chandraprakash
One use case I see for setting the `segment.bytes` to 1 is to delete all the records from the topic. We can mention about it in the doc to use the `kafka-delete-records` API instead. On Wed, Mar 13, 2024 at 6:59 PM Divij Vaidya wrote: > + users@kafka > > Hi users of Apache Kafka > > With the

Re: [VOTE] KIP-956: Tiered Storage Quotas

2024-03-08 Thread Kamal Chandraprakash
+1 (non-binding), Thanks for the KIP, Abhijeet! -- Kamal On Fri, Mar 8, 2024 at 11:02 PM Jun Rao wrote: > Hi, Abhijeet, > > Thanks for the KIP. +1 > > Jun > > On Fri, Mar 8, 2024 at 3:44 AM Abhijeet Kumar > wrote: > > > Hi All, > > > > I would like to start the vote for KIP-956 - Tiered Storag

Re: [ANNOUNCE] Apache Kafka 3.7.0

2024-02-29 Thread Kamal Chandraprakash
Alok Thatikunta, Alyssa Huang, Aman Singh, Andras Katona, Andrew > > > > Schofield, Anna Sophie Blee-Goldman, Anton Agestam, Apoorv Mittal, > > > > Arnout Engelen, Arpit Goyal, Artem Livshits, Ashwin Pankaj, > > > > ashwinpankaj, atu-sharm, bachmanit

Re: [DISCUSS] KIP-1018: Introduce max remote fetch timeout config

2024-02-01 Thread Kamal Chandraprakash
pache.org/jira/browse/KAFKA-15776 > [2] https://github.com/apache/kafka/pull/14778#issuecomment-1820588080 > > On Tue, 30 Jan 2024 at 01:01, Kamal Chandraprakash < > kamal.chandraprak...@gmail.com> wrote: > > > Hi all, > > > > I have opened a KIP-1018 > > &

[DISCUSS] KIP-1018: Introduce max remote fetch timeout config

2024-01-29 Thread Kamal Chandraprakash
Hi all, I have opened a KIP-1018 to introduce dynamic max-remote-fetch-timeout broker config to give more control to the operator. https://cwiki.apache.org/con

Re: [VOTE] KIP-1011: Use incrementalAlterConfigs when updating broker configs by kafka-configs.sh

2024-01-21 Thread Kamal Chandraprakash
+1 (non-binding) On Mon, Jan 22, 2024 at 8:34 AM ziming deng wrote: > Hello everyone, > I'd like to initiate a vote for KIP-1011. > This KIP is about replacing alterConfigs with incrementalAlterConfigs when > updating broker configs using kafka-configs.sh, this is similar to what we > have done

[jira] [Created] (KAFKA-16146) Checkpoint log-start-offset after remote log deletion

2024-01-16 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-16146: Summary: Checkpoint log-start-offset after remote log deletion Key: KAFKA-16146 URL: https://issues.apache.org/jira/browse/KAFKA-16146 Project: Kafka

Re: [VOTE] KIP-1007: Introduce Remote Storage Not Ready Exception

2024-01-07 Thread Kamal Chandraprakash
as it will be a more explicit > > > > behaviour than what Kafka does today - I am happy with it! > > > > > > > > +1 (non-binding) from me > > > > > > > > Best, > > > > Christo > > > > > > > > On Tue, 1

Re: [VOTE] KIP-1013: Drop broker and tools support for Java 11 in Kafka 4.0 (deprecate in 3.7)

2024-01-02 Thread Kamal Chandraprakash
+1 (non-binding). On Wed, Jan 3, 2024 at 8:01 AM Satish Duggana wrote: > Thanks Ismael for the proposal. > > Adopting JDK 17 enhances developer productivity and has reached a > level of maturity that has led to its adoption by several other major > projects, signifying its reliability and effect

Re: [ANNOUNCE] New Kafka PMC Member: Divij Vaidya

2023-12-27 Thread Kamal Chandraprakash
Congrats Divij! On Thu, Dec 28, 2023, 07:09 Kirk True wrote: > Congrats Divij!!! > > On Wed, Dec 27, 2023, at 1:44 PM, Jorge Esteban Quilcate Otoya wrote: > > Congratulations Divij!! > > > > On Wed 27. Dec 2023 at 14.56, Tom Bentley wrote: > > > > > Congratulations! > > > > > > On Thu, 28 Dec 2

Re: [VOTE] KIP-1005: Expose EarliestLocalOffset and TieredOffset

2023-12-25 Thread Kamal Chandraprakash
+1 (non-binding). Thanks for the KIP! -- Kamal On Thu, Dec 21, 2023 at 2:23 PM Christo Lolov wrote: > Heya all! > > KIP-1005 ( > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1005%3A+Expose+EarliestLocalOffset+and+TieredOffset > ) > has been open for around a month with no further com

Re: DISCUSS KIP-1011: Use incrementalAlterConfigs when updating broker configs by kafka-configs.sh

2023-12-25 Thread Kamal Chandraprakash
Hi Ziming, Thanks for the KIP! The proposal LGTM. I'm also inclined towards option 2 (i.e. add an explicit --enable-incremental flag in 3.X version) to avoid any incompatible change in v3.X. As mentioned in this thread, many users might be using external tools to do the topic rebalance and apply

Re: [DISCUSS] KIP-1007: Introduce Remote Storage Not Ready Exception

2023-12-25 Thread Kamal Chandraprakash
d on a partition. > > [1] > > https://github.com/apache/kafka/blob/82808873cbf6a95611243c2e7984c4aa6ff2cfff/core/src/main/scala/kafka/log/UnifiedLog.scala#L1336 > > > -- > Divij Vaidya > > > > On Tue, Dec 12, 2023 at 9:58 AM Kamal Chandraprakash < > kamal.chandraprak...@gmail.com> wrote: &

[VOTE] KIP-1007: Introduce Remote Storage Not Ready Exception

2023-12-12 Thread Kamal Chandraprakash
Hi, I would like to call a vote for KIP-1007 . This KIP aims to introduce a new error code for retriable remote storage errors. Thanks to everyone who reviewed the KIP! -- Kamal

Re: [DISCUSS] KIP-1007: Introduce Remote Storage Not Ready Exception

2023-12-12 Thread Kamal Chandraprakash
Thanks Luke for reviewing this KIP! If there are no more comments from others, I'll start the VOTE since this is a minor KIP. On Mon, Dec 11, 2023 at 1:01 PM Luke Chen wrote: > Hi Kamal, > > Thanks for the KIP! > LGTM. > > Thanks. > Luke > > On Wed, Nov 22, 2023

Re: [VOTE] 3.6.1 RC0

2023-11-27 Thread Kamal Chandraprakash
+1 (non-binding) 1. Built the source from 3.6.1-rc0 tag 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. On Tue, Nov 28, 2023 at 12:55 A

[DISCUSS] KIP-1007: Introduce Remote Storage Not Ready Exception

2023-11-22 Thread Kamal Chandraprakash
Hi, I would like to start a discussion to introduce a new error code for retriable remote storage errors. Please take a look at the proposal: https://cwiki.apache.org/confluence/display/KAFKA/KIP-1007%3A+Introduce+Remote+Storage+Not+Ready+Exception

[jira] [Created] (KAFKA-15876) Introduce Remote Storage Not Ready Exception

2023-11-22 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-15876: Summary: Introduce Remote Storage Not Ready Exception Key: KAFKA-15876 URL: https://issues.apache.org/jira/browse/KAFKA-15876 Project: Kafka

Re: [VOTE] KIP-963: Additional metrics in Tiered Storage

2023-11-20 Thread Kamal Chandraprakash
+1 (non-binding). Thanks for the KIP! On Tue, Nov 21, 2023, 03:04 Divij Vaidya wrote: > + 1 (binding) > > This Kip will greatly improve Tiered Storage troubleshooting. Thank you > Christo. > > On Mon 20. Nov 2023 at 17:21, Christo Lolov > wrote: > > > Hello all! > > > > Now that the discussion

[jira] [Created] (KAFKA-15859) Introduce delayed remote list offsets to make LIST_OFFSETS async

2023-11-20 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-15859: Summary: Introduce delayed remote list offsets to make LIST_OFFSETS async Key: KAFKA-15859 URL: https://issues.apache.org/jira/browse/KAFKA-15859

Re: [DISCUSS] KIP-1005: Add EarliestLocalOffset to GetOffsetShell

2023-11-20 Thread Kamal Chandraprakash
Hi Christo, Thanks for the KIP! Similar to the earliest-local-log offset, can we also expose the highest-copied-remote-offset via GetOffsetShell tool? This will be useful during the debugging session. On Mon, Nov 20, 2023 at 5:38 PM Christo Lolov wrote: > Hello all! > > I would like to start

Re: RemoteStorageManager.fetchLogSegment - how to deal with InterruptedException?

2023-11-17 Thread Kamal Chandraprakash
Hi Ivan, I've opened a relevant patch to increase the timeout to fetch data from remote storage. This will reduce the error occurrence rate: https://github.com/apache/kafka/pull/14778 I think it's better to distinguish the cases: 1) Any error that happens while reading the remote data -- this s

Re: [VOTE] KIP-997: Partition-Level Throughput Metrics

2023-11-16 Thread Kamal Chandraprakash
+1 (non-binding). Thanks for the KIP! On Thu, Nov 16, 2023 at 9:00 AM Satish Duggana wrote: > Thanks Qichao for the KIP. > > +1 (binding) > > ~Satish. > > On Thu, 16 Nov 2023 at 02:20, Jorge Esteban Quilcate Otoya > wrote: > > > > Qichao, thanks again for leading this proposal! > > > > +1 (non-

[jira] [Resolved] (KAFKA-15376) Explore options of removing data earlier to the current leader's leader epoch lineage for topics enabled with tiered storage.

2023-11-14 Thread Kamal Chandraprakash (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15376?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Kamal Chandraprakash resolved KAFKA-15376. -- Resolution: Fixed This task was already addressed in the code, so closing the

[jira] [Created] (KAFKA-15777) Configurable remote fetch bytes per partition from Consumer

2023-11-02 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-15777: Summary: Configurable remote fetch bytes per partition from Consumer Key: KAFKA-15777 URL: https://issues.apache.org/jira/browse/KAFKA-15777 Project

[jira] [Created] (KAFKA-15776) Configurable delay timeout for DelayedRemoteFetch request

2023-11-02 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-15776: Summary: Configurable delay timeout for DelayedRemoteFetch request Key: KAFKA-15776 URL: https://issues.apache.org/jira/browse/KAFKA-15776 Project

Re: [DISCUSS] KIP-963: Upload and delete lag metrics in Tiered Storage

2023-11-02 Thread Kamal Chandraprakash
> > > > On Mon, 25 Sept 2023 at 10:49, Gantigmaa Selenge < > > gsele...@redhat.com> > > > > > wrote: > > > > > > > > > >> Hi Christo, > > > > >> > > > > >> Thank you for writing the KIP.

  1   2   3   >