world.
I am ok with adding a note in the KIP but the note should say that it has
an elevated risk for this scenario due to increased probability of having
an aggressive local cleanup with Tiered Storage.
--
Divij Vaidya
On Wed, Jul 24, 2024 at 1:22 PM Abhijeet Kumar
wrote:
> Hi Divij,
>
gt; available anymore, the follower will eventually be able to catch up with
> the leader by resetting its fetch offset until the offset
> is available on the leader's local log. Once it catches up, replication on
> the follower can proceed.
>
> Regards,
> Abhijeet.
>
>
>
te to respond to
the follower. The amount of this data can be controlled via the configured
value Z which can be set based on how aggressive the upload/archival
process is.
--
Divij Vaidya
On Tue, Jul 2, 2024 at 12:25 PM Divij Vaidya
wrote:
> Hi folks.
>
> I am late to the party bu
OffsetMovedToTieredException
3. Follower builds aux state from [0-10] and sets the fetch offset to 11
4. But leader has already uploaded more data and now the new
last-tiered-offset is 15
5. Go back to 2
This could cause a cycle where the replica will be stuck trying to
reconcile with the leader.
--
Divij Vaidya
On Fri
+1 (binding)
I have participated in the discussion and agree with the proposal.
--
Divij Vaidya
On Thu, Jun 27, 2024 at 12:56 PM Satish Duggana
wrote:
> Thanks Federico for the KIP.
>
> +1
>
> ~Satish.
>
> On Thu, 27 Jun 2024 at 13:44, Federico Valeri
> wrote:
>
Thank you Federico for answering the questions. No more questions/concerns
from me. The KIP looks good.
--
Divij Vaidya
On Wed, Jun 26, 2024 at 11:02 AM Federico Valeri
wrote:
> Hi Divij, thanks for you questions and suggestions, much appreciated.
>
> On Tue, Jun 25, 2024 at 1:12
ecify a particular schema version?
--
Divij Vaidya
On Thu, Jun 20, 2024 at 5:57 PM Kamal Chandraprakash <
kamal.chandraprak...@gmail.com> wrote:
> Hi Federico,
>
> Thanks for the KIP! +1 from me.
>
> On Wed, Jun 19, 2024, 17:36 Luke Chen wrote:
>
> > Hi Federico,
&
ve 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:
> Bump. Please review this proposal.
>
>
> On Mon, Jun 17
oposed API take "segmentId" as a parameter 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 f
cer buffer) but
we have started recording end-to-end latency for the steady-state messages.
I imagine this should be ok because it achieves the purpose of
removing bootstrap times, but I haven't been able to reason about it in my
head. What are your thoughts on this?
--
Divij Vaidya
On Fr
ibility will be delegated to the decoder class which the
operator can define using the flag "--value-decoder-class".
Is there a reason that you are unable to use the rejected solution (which
requires no changes) for debugging purposes?
--
Divij Vaidya
On Sat, Jun 15, 2024 at 4:43 PM Federico
approach (and other alternatives) to
solve this issue?
—
Divij Vaidya
On Thu 16. May 2024 at 22:15, Jorge Esteban Quilcate Otoya <
quilcate.jo...@gmail.com> wrote:
> Thanks Justine. I have updated the KIP with the configuration details.
>
> On Thu, 16 May 2024 at 21:14, Justine Ol
,
just to get a better aligned conventional does not seem worthwhile to me.
Is there an obvious benefit that I am missing here which would make this
proposal a good trade off with the cost?
—
Divij Vaidya
On Thu 6. Jun 2024 at 21:13, Eric Lu wrote:
> Hi,
>
> I wanted to follow-
Congratulations Greg!
--
Divij Vaidya
On Sun, Apr 14, 2024 at 6:39 AM Kamal Chandraprakash <
kamal.chandraprak...@gmail.com> wrote:
> Congratulations, Greg!
>
> On Sun, Apr 14, 2024 at 8:57 AM Yash Mayya wrote:
>
> > Congrats Greg!
> >
> > On Sun, 14 Ap
Hey Manikumar
Are we planning to add the blog entry about the 3.6.2 release at
https://kafka.apache.org/blog ? Asking because I didn't see this included
in the release announcement.
--
Divij Vaidya
On Wed, Mar 20, 2024 at 12:01 PM Manikumar
wrote:
> Hi,
>
> We have one non-bl
zstd)
Things look good to me. We don't need another RC for fixing docs.
+1 (binding) from me.
--
Divij Vaidya
On Thu, Apr 4, 2024 at 10:04 AM Manikumar wrote:
> Hi Justine,
>
> Thanks for catching this. looks like we have missed updating
> `docs/documentation.html` in kafka re
.1%20ORDER%20BY%20priority%20DESC%2C%20updated%20DESC
--
Divij Vaidya
On Thu, Mar 14, 2024 at 7:55 AM Manikumar wrote:
> Hi all,
>
> Here is the release plan for 3.6.2:
> https://cwiki.apache.org/confluence/display/KAFKA/Release+plan+3.6.2
>
> Currently there is one open non-
+1
Thank you for volunteering.
--
Divij Vaidya
On Wed, Mar 13, 2024 at 4:58 PM Justine Olshan
wrote:
> Thanks Manikumar!
> +1 from me
>
> Justine
>
> On Wed, Mar 13, 2024 at 8:52 AM Manikumar
> wrote:
>
> > Hi,
> >
> > I'd like to volunteer
adjusted. Please reply in this
thread directly.
--
Divij Vaidya
Apache Kafka PMC
On Wed, Mar 13, 2024 at 12:56 PM Divij Vaidya
wrote:
> Thanks for the discussion folks. I have started a KIP
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1030%3A+Change+constraints+and+default+valu
once it is ready to be
published, we can start a discussion thread on it.
I am also going to start a thread to solicit feedback from users@ mailing
list as well.
--
Divij Vaidya
On Wed, Mar 13, 2024 at 12:55 PM Christopher Shannon <
christopher.l.shan...@gmail.com> wrote:
> I thi
Divij Vaidya created KAFKA-16368:
Summary: Change constraints and default values for various
configurations
Key: KAFKA-16368
URL: https://issues.apache.org/jira/browse/KAFKA-16368
Project: Kafka
t want to work
with a segment.ms of less than ~10s (number chosen arbitrary that looks
sane)
What was the historical context of setting the minimum threshold to 1ms for
this setting?
[1] https://kafka.apache.org/documentation.html#topicconfigs_segment.ms
--
Divij Vaidya
[
https://issues.apache.org/jira/browse/KAFKA-16126?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Divij Vaidya resolved KAFKA-16126.
--
Resolution: Fixed
> Kcontroller dynamic configurations may fail to apply at star
. You can either create a KIP
offline and share it in the discussion email or I can create the KIP page
on your behalf and copy/paste the content that you send to me.
--
Divij Vaidya
On Thu, Mar 7, 2024 at 5:15 PM Arpit Goyal wrote:
> + @Kamal Chandraprakash @Vaidya, Divij
> Can you h
Divij Vaidya created KAFKA-16325:
Summary: Add missing producer metrics to documentation
Key: KAFKA-16325
URL: https://issues.apache.org/jira/browse/KAFKA-16325
Project: Kafka
Issue Type
ful changes in this release but the one closest to my
heart is https://issues.apache.org/jira/browse/KAFKA-15046. I am very glad
this is fixed. The P999 latency spikes were driving me crazy for a long
time now.
--
Divij Vaidya
On Wed, Feb 28, 2024 at 10:06 AM Satish Duggana
wrote:
> Thanks Sta
ntral (mvnrepository.com or maven.org)."
in the release process wiki.
--
Divij Vaidya
On Tue, Feb 27, 2024 at 4:43 PM Stanislav Kozlovski
wrote:
> Hey all,
>
> Everything site-related is merged.
>
> I have been following the final steps of the release process.
> - Docker
Thank you for volunteering Josep. +1 from me.
--
Divij Vaidya
On Tue, Feb 27, 2024 at 9:35 AM Bruno Cadonna wrote:
> Thanks Josep!
>
> +1
>
> Best,
> Bruno
>
> On 2/26/24 9:53 PM, Chris Egerton wrote:
> > Thanks Josep, I'm +1 as well.
> >
> >
Hey Vaibhav
I have provided you with contributor permission to the JIRA. You should be
able to assign the JIRA to yourself now.
--
Divij Vaidya
On Sun, Feb 25, 2024 at 12:23 AM Vaibhav Kushwaha
wrote:
> Hi team!
>
> I was going through the list of starter bugs and found one I coul
e your PRs
on kafka-site repo get merged.
4. Send a release announcement by email.
--
Divij Vaidya
On Tue, Feb 20, 2024 at 3:02 PM Stanislav Kozlovski
wrote:
> Thanks for testing the release! And thanks for the review on the
> documentation. Good catch on the license too.
>
> I have
kafka binaries.
I am +1 (binding) assuming Mickael's concerns about update notes to 3.7 are
addressed before release.
--
Divij Vaidya
On Mon, Feb 19, 2024 at 6:08 PM Mickael Maison
wrote:
> Hi,
>
> I agree with Josep, I don't think it's worth making a new RC j
Divij Vaidya created KAFKA-16278:
Summary: Missing license for scala related dependencies
Key: KAFKA-16278
URL: https://issues.apache.org/jira/browse/KAFKA-16278
Project: Kafka
Issue Type
test results
Test: Spot check the results of system tests.
Result: I have verified that the system tests are passing across different
runs.
--
Divij Vaidya
On Sun, Feb 18, 2024 at 2:50 PM Stanislav Kozlovski
wrote:
> The latest system test build completed successfully -
>
> https
[
https://issues.apache.org/jira/browse/KAFKA-16239?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Divij Vaidya resolved KAFKA-16239.
--
Resolution: Fixed
> Clean up references to non-existent IntegrationTestHel
[
https://issues.apache.org/jira/browse/KAFKA-14041?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Divij Vaidya resolved KAFKA-14041.
--
Resolution: Fixed
> Avoid the keyword var for a variable declaration in ConfigTransfor
Divij Vaidya created KAFKA-16239:
Summary: Clean up references to non-existent IntegrationTestHelper
Key: KAFKA-16239
URL: https://issues.apache.org/jira/browse/KAFKA-16239
Project: Kafka
Sounds good. I am onboard to start with first steps and eventually move
towards a place where compression codec settings are more generic /
pluggable.
--
Divij Vaidya
On Wed, Feb 7, 2024 at 3:40 PM Mickael Maison
wrote:
> Hi Divij,
>
> Thanks for bringing that point. After reading K
compression controls in Kafka?
It might be beneficial to have a joint proposal of these two KIPs which may
satisfy both use cases.
--
Divij Vaidya
On Wed, Feb 7, 2024 at 11:14 AM Mickael Maison
wrote:
> Hi,
>
> I'm resurrecting this old thread as this KIP would be a nice
> improve
ed+Storage+Early+Access+Release+Notes
--
Divij Vaidya
On Thu, Jan 11, 2024 at 1:03 PM Luke Chen wrote:
> Hi all,
>
> There is a bug KAFKA-16101
> <https://issues.apache.org/jira/browse/KAFKA-16101> reporting that "Kafka
> cluster will be unavailable during KRaft migrat
[
https://issues.apache.org/jira/browse/KAFKA-9693?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Divij Vaidya resolved KAFKA-9693.
-
Resolution: Fixed
This performance regression where producer snapshot fsync leads to high P99
[
https://issues.apache.org/jira/browse/KAFKA-16210?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Divij Vaidya resolved KAFKA-16210.
--
Resolution: Fixed
> Upgrade jose4j to 0.9.4
> ---
>
>
Divij Vaidya created KAFKA-16210:
Summary: Upgrade jose4j to 0.9.4
Key: KAFKA-16210
URL: https://issues.apache.org/jira/browse/KAFKA-16210
Project: Kafka
Issue Type: Improvement
+1 (binding)
I have participated in the discussion for this and looked at the most
recent version of this KIP. It looks good to me.
--
Divij Vaidya
On Tue, Jan 23, 2024 at 8:17 AM David Jacot
wrote:
> Hi Chris, Ziming,
>
> Thanks for the clarification. I am glad that it does not i
+1 (binding)
Divij Vaidya
On Tue, Dec 26, 2023 at 7:05 AM Kamal Chandraprakash <
kamal.chandraprak...@gmail.com> wrote:
> +1 (non-binding). Thanks for the KIP!
>
> --
> Kamal
>
> On Thu, Dec 21, 2023 at 2:23 PM Christo Lolov
> wrote:
>
> > Heya all
Thank you for making the change Christo. It looks good to me.
--
Divij Vaidya
On Thu, Jan 11, 2024 at 11:19 AM Christo Lolov
wrote:
> Thank you Divij!
>
> I have updated the KIP to explicitly state that the broker will have a
> different behaviour when a timestamp of -5 is reque
/pages/viewpage.action?pageId=284789510
Does this answer your question?
--
Divij Vaidya
On Wed, Jan 10, 2024 at 9:37 PM Devinder Saggu
wrote:
> Hi,
>
> I wonder how long Kafka 3.0 can support Java 8.
>
> Thanks & Regards,
>
> *Devinder Singh*
> P *Please consider the environment before printing this email*
>
her to add them or not? If we keep a blanket criteria of
accepting all PRs, then we may end up in a situation where the llink
redirects to a phishing page or nefarious website. Hence, we might have to
at least perform some basic due diligence which adds overhead to the
resources of the community.
ith AWS & Apache Infra in the
next 1-2 months.
--
Divij Vaidya
On Tue, Jan 9, 2024 at 9:21 PM Colin McCabe wrote:
> Sorry, but to put it bluntly, the current build setup isn't good enough at
> partial rebuilds that build caching would make sense. All Kafka devs have
> had t
Thank you for addressing my concerns Kamal. Though, instead of the KIP, I
actually was suggesting to add it in JavaDoc so that someone looking at the
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
+1 (binding)
--
Divij Vaidya
On Thu, Dec 21, 2023 at 10:30 AM Luke Chen wrote:
> Hi Kamal,
>
> Thanks for the KIP.
> +1 (binding) from me.
>
> Luke
>
> On Thu, Dec 21, 2023 at 4:51 PM Christo Lolov
> wrote:
>
> > Heya Kamal,
> >
> > The propos
+1 (binding)
--
Divij Vaidya
On Wed, Jan 3, 2024 at 11:06 AM Viktor Somogyi-Vass
wrote:
> Hi Ismael,
>
> I think it's important to make this change, the youtube video you posted on
> the discussion thread makes very good arguments and so does the KIP. Java 8
> is almost
Divij Vaidya created KAFKA-16074:
Summary: Fix thread leaks in ReplicaManagerTest
Key: KAFKA-16074
URL: https://issues.apache.org/jira/browse/KAFKA-16074
Project: Kafka
Issue Type: Sub-task
interfaces in the KIP.
--
Divij Vaidya
On Tue, Nov 21, 2023 at 2:19 PM Christo Lolov
wrote:
> Heya!
>
> Thanks a lot for this. I have updated the KIP to include exposing the
> tiered-offset as well. Let me know whether the Public Interfaces section
> needs more explanation
Divij Vaidya created KAFKA-16072:
Summary: Create Mockito extension to detect thread leak
Key: KAFKA-16072
URL: https://issues.apache.org/jira/browse/KAFKA-16072
Project: Kafka
Issue Type
Divij Vaidya created KAFKA-16066:
Summary: Upgrade apacheds to 2.0.0.AM27
Key: KAFKA-16066
URL: https://issues.apache.org/jira/browse/KAFKA-16066
Project: Kafka
Issue Type: Improvement
Divij Vaidya created KAFKA-16063:
Summary: Fix leaked ApplicationShutdownHooks in
EndToEndAuthorizationTests
Key: KAFKA-16063
URL: https://issues.apache.org/jira/browse/KAFKA-16063
Project: Kafka
Divij Vaidya created KAFKA-16062:
Summary: Upgrade mockito to 5.8.0
Key: KAFKA-16062
URL: https://issues.apache.org/jira/browse/KAFKA-16062
Project: Kafka
Issue Type: Sub-task
[
https://issues.apache.org/jira/browse/KAFKA-16060?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Divij Vaidya resolved KAFKA-16060.
--
Resolution: Not A Problem
> Some questions about tiered storage capabilit
[
https://issues.apache.org/jira/browse/KAFKA-16015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Divij Vaidya resolved KAFKA-16015.
--
Resolution: Fixed
> kafka-leader-election timeout values always overwritten by default val
Thank you everyone for your warm wishes 🙏
--
Divij Vaidya
On Thu, Dec 28, 2023 at 2:37 PM Yash Mayya wrote:
> Congratulations Divij!
>
> On Wed, Dec 27, 2023 at 5:15 PM Luke Chen wrote:
>
> > Hi, Everyone,
> >
> > Divij has been a Kafka committer since Ju
Divij Vaidya created KAFKA-16059:
Summary: Fix trhead l
Key: KAFKA-16059
URL: https://issues.apache.org/jira/browse/KAFKA-16059
Project: Kafka
Issue Type: Sub-task
Reporter
I have started to perform an analysis of the OOM at
https://issues.apache.org/jira/browse/KAFKA-16052. Please feel free to
contribute to the investigation.
--
Divij Vaidya
On Wed, Dec 27, 2023 at 1:23 AM Justine Olshan
wrote:
> I am still seeing quite a few OOM errors in the builds and I
Divij Vaidya created KAFKA-16053:
Summary: Fix leaked Default DirectoryService
Key: KAFKA-16053
URL: https://issues.apache.org/jira/browse/KAFKA-16053
Project: Kafka
Issue Type: Sub-task
Divij Vaidya created KAFKA-16052:
Summary: OOM in Kafka test suite
Key: KAFKA-16052
URL: https://issues.apache.org/jira/browse/KAFKA-16052
Project: Kafka
Issue Type: Bug
Affects Versions
anecdote, a recent performance regression was found in 17,
https://bugs.openjdk.org/browse/JDK-8317960, which was already fixed in 21)
--
Divij Vaidya
On Tue, Dec 26, 2023 at 9:58 PM Ismael Juma wrote:
> Hi Colin,
>
> A couple of comments:
>
> 1. It is true that full support for
, this incompatibility will be part of
one amongst many changes that users will perform to upgrade to 4.x. This is
unlike a major version change from 3.7 to 3.8 where users expect a simple
upgrade without any code changes.
Let's wait and hear what others think about this.
--
Divij Vaidya
O
[
https://issues.apache.org/jira/browse/KAFKA-16021?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Divij Vaidya resolved KAFKA-16021.
--
Resolution: Fixed
> Eagerly load Charset for StringSerializer/StringDeseriali
Divij Vaidya created KAFKA-16048:
Summary: Test failure
kafka.admin.ConfigCommandTest.shouldNotSupportAlterClientMetricsWithZookeeper
Key: KAFKA-16048
URL: https://issues.apache.org/jira/browse/KAFKA-16048
shared here.
--
Divij Vaidya
On Fri, Dec 22, 2023 at 2:05 PM Luke Chen wrote:
> Hi Sophie and Philip and all,
>
> I share the same pain as you.
> I've been waiting for a CI build result in a PR for days. Unfortunately, I
> can only get 1 result each day because it takes 8 hou
error
means for operations that can be performed 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..
on and compression that occurs during message
version transformation, i.e. when client send message with V1 and broker
expects in V2, we convert the message and recompress it.
--
Divij Vaidya
On Mon, Dec 18, 2023 at 7:22 PM Diop, Assane wrote:
> I would like to bring some attention to
r upgrade on a
Kafka version. From 4.0, we will of course, default to using
incrementalAlterConfig.
--
Divij Vaidya
On Fri, Dec 22, 2023 at 6:54 AM ziming deng
wrote:
> +1 for adding them to rejected alternatives, These kafka-ui tools should
> also evolve with the iterations of Kafk
ous
other things (I came across another one
https://issues.apache.org/jira/browse/KAFKA-16041) that we can start doing
now for 4.x.
What do you think?
--
Divij Vaidya
On Thu, Dec 21, 2023 at 4:30 PM David Jacot
wrote:
> Hi Divij,
>
> > Release 4.0 as an "experimental" release
tely new functionality.
[1] https://issues.apache.org/jira/browse/KAFKA-15495
[2] https://issues.apache.org/jira/browse/KAFKA-15489
[3] https://issues.apache.org/jira/browse/KAFKA-14874
--
Divij Vaidya
On Wed, Dec 20, 2023 at 4:59 PM Josep Prat
wrote:
> Hi Justine, Luke, and others,
>
+ 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 for KIP-963 has winded down, I would like to open
> it for a vote targeting 3.7.0 as the release. You can fi
[
https://issues.apache.org/jira/browse/KAFKA-15481?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Divij Vaidya resolved KAFKA-15481.
--
Resolution: Fixed
> Concurrency bug in RemoteIndexCache leads to IOExcept
+1 (binding)
I was involved in the discussion thread for this KIP and support it in its
current form.
--
Divij Vaidya
On Wed, Nov 15, 2023 at 10:55 AM Qichao Chu wrote:
> Hi all,
>
> I'd like to call a vote for KIP-977: Partition-Level Throughput Metrics.
>
> Ple
[
https://issues.apache.org/jira/browse/KAFKA-15658?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Divij Vaidya resolved KAFKA-15658.
--
Resolution: Fixed
> Zookeeper.jar | CVE-2023-44
ny further questions regarding this, please don't hesitate to
reach out to us at secur...@kafka.apache.org or post a comment at
https://issues.apache.org/jira/browse/KAFKA-15658
Best Regards,
Divij Vaidya
On behalf of Apache Kafka PMC
[1] https://zookeeper.apache.org/security.html#CVE-2023-
[
https://issues.apache.org/jira/browse/KAFKA-15376?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Divij Vaidya reopened KAFKA-15376:
--
Assignee: (was: Satish Duggana)
> Explore options of removing data earlier to
1]
https://github.com/apache/kafka/blob/832627fc78484fdc7c8d6da8a2d20e7691dbf882/core/src/main/java/kafka/log/remote/RemoteLogManager.java#L1310
[2]
https://github.com/apache/kafka/blob/832627fc78484fdc7c8d6da8a2d20e7691dbf882/core/src/main/java/kafka/log/remote/RemoteLogManager.java#L1358
--
Di
s://kafka.apache.org/contributing -> Finding a project to work on
Divij Vaidya
On Mon, Nov 13, 2023 at 4:24 PM Николай Ижиков wrote:
>
> > To kickstart this effort, we can publish a list of such tickets in the
> community and assign one or more committers the role of a «shepherd&quo
ffort to
merge some of the critical ones by next week. I will nudge the contributors
where things are pending for a while.
[1] https://issues.apache.org/jira/browse/KAFKA-15420
--
Divij Vaidya
On Mon, Nov 13, 2023 at 4:10 PM Mickael Maison
wrote:
> Hi Divij,
>
> You beat me to i
o,
my offer to volunteer for Jan release could be considered as a backup.
--
Divij Vaidya
On Mon, Nov 13, 2023 at 3:40 PM Ismael Juma wrote:
> Hi Divij,
>
> I think we should be releasing 3.6.1 this year rather than next. There are
> some critical bugs in 3.6.0 and I don't thin
As a future step in Tiered Storage, I would actually
prefer to move towards a direction where we are lazily fetching indexes
on-demand instead of fetching them together as proposed in the KIP.
--
Divij Vaidya
On Fri, Nov 10, 2023 at 4:00 PM Jorge Esteban Quilcate Otoya <
quilcate.jo...@gmail.co
e are targeting for 3.6.1 along with
the detailed timeline.
If anyone is interested in releasing this sooner, please feel free to take
over from me.
Thanks!
Regards,
Divij Vaidya
Apache Kafka Committer
[1] https://issues.apache.org/jira/browse/KAFKA-15653
[2] https://issues.apache.org/jira/browse/
Thank you for updating the KIP Qichao.
I don't have any more questions or suggestions. Looks good to move forward
from my perspective.
--
Divij Vaidya
On Fri, Nov 10, 2023 at 2:25 PM Qichao Chu wrote:
> Thank you again for the nice suggestions, Jorge!
> I will wait for Divij
viewers in allocating their bandwidth effectively.
--
Divij Vaidya
On Sun, Nov 12, 2023 at 2:58 AM Justine Olshan
wrote:
> I will say that I have also seen tests that seem to be more flaky
> intermittently. It may be ok for some time and suddenly the CI is
> overloaded and we see is
f
not, RemoteLogSegmentMetadata also contains a map, CustomMetadata which is
opaque to the broker and is populated by RemoteLogMetadataManager. You can
choose to feed any attributes you require into this custom metadata and
read it in RSM.
Does this answer your question?
--
Divij Vaidya
On Wed, Nov 8, 20
time to think about the implications of current approach towards
extendibility to other dimensions in future (such as e.g. client language
instead of partition).
I will get back to you before the end of this week.
--
Divij Vaidya
On Tue, Nov 7, 2023 at 4:05 PM Qichao Chu wrote:
> Hi Di
[
https://issues.apache.org/jira/browse/KAFKA-15771?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Divij Vaidya resolved KAFKA-15771.
--
Resolution: Fixed
> ProduceRequest#partitionSizes() is not an atomic operat
Divij Vaidya created KAFKA-15793:
Summary: Flaky test
ZkMigrationIntegrationTest.testMigrateTopicDeletions
Key: KAFKA-15793
URL: https://issues.apache.org/jira/browse/KAFKA-15793
Project: Kafka
[
https://issues.apache.org/jira/browse/KAFKA-15769?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Divij Vaidya resolved KAFKA-15769.
--
Resolution: Fixed
> Fix wrong log with except
Hey Dasun
Subscribe/unsubscribe to the mailing list is self service. You can find the
details at https://kafka.apache.org/contact
--
Divij Vaidya
On Fri, Nov 3, 2023 at 7:37 PM Dasun Nirmitha
wrote:
> Hello Apache Kafka Mail Thread,
> Thanks for having me in your mail thread for a
Divij Vaidya created KAFKA-15772:
Summary: Flay test TransactionsWithTieredStoreTest
Key: KAFKA-15772
URL: https://issues.apache.org/jira/browse/KAFKA-15772
Project: Kafka
Issue Type: Test
ell)
4. what happens when there are contradictory regEx rules, e.g. a topic
defined in high as well as low. It is generally solved by defining
precedence. In our case, we can choose that high has more precedence than
low.
What do you think?
--
Divij Vaidya
On Wed, Nov 1, 2023 at 2:07 PM Qichao Chu
Divij Vaidya created KAFKA-15764:
Summary: Missing tests for transactions
Key: KAFKA-15764
URL: https://issues.apache.org/jira/browse/KAFKA-15764
Project: Kafka
Issue Type: Test
r? Also, I am happy with having a Kafka level
dynamic config that we can use to filter our metric/dimensionality since we
have a precedence at KIP-544. Hence, my suggestion to push this filtering
to metric library can be ignored.
--
Divij Vaidya
On Sat, Oct 28, 2023 at 11:37 AM Qichao Ch
Congratulations Satish! And thank you for your contributions so far.
--
Divij Vaidya
On Fri, Oct 27, 2023 at 5:18 PM Lucas Brutschy
wrote:
> Congrats!
>
> On Fri, Oct 27, 2023 at 5:06 PM Manikumar
> wrote:
> >
> > Congrats!
> >
> > On Fri, Oct
[
https://issues.apache.org/jira/browse/KAFKA-15685?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Divij Vaidya resolved KAFKA-15685.
--
Resolution: Fixed
> Missing compatibility for MinGW (wind
1 - 100 of 380 matches
Mail list logo