Hi Mickael,
KIP-891 [1] is/was merged and feature complete before the 4.1 branch was
cut.
And the JIRA was resolved [2] but wasn't given a fix version (oops) so it's
not currently in the Release Plan.
Can I add KIP-891 to the Release Plan?
Thanks,
Greg
[1]
https://cwiki.apache.org/confluence/di
eferable? The choice would determine what security
> mitigations are needed if any.
>
> https://cwiki.apache.org/confluence/x/Xgr0Ew
> Thanks,
> Olasoji
>
> -Original Message-
> From: Greg Harris
> Sent: Monday, February 24, 2025 12:42 PM
> To: dev@kafka.apache
Hi Pritam,
Thanks for the KIP!
I'm a little unsure of the motivation here and would appreciate some more
context from your experience.
1. The KIP states "Broker resource utilization is expected to decrease by
approximately 20%, primarily due to reduced partition count and metadata
overhead." Can
Hi Claude,
Please see the methods on the Embedded connect class,
#configureConnector(String, Map) and similar.
If you specify different connector IDs, this will create multiple
concurrent connectors.
Thanks,
Greg
On Mon, Apr 14, 2025, 7:45 AM Claude Warren, Jr
wrote:
> Greetings,
>
> I am look
Hi All,
I am still strongly in support of backporting this patch as I stated in the
earlier discussion thread.
Thank you Stig, Anton, Danish, Mateusz, Manfred, Monica, Istvan, Vincent,
Clement, Anshu, Foivos, Drakgoku, Severin, Ozan, Georgios, and Guillaume,
members of our extended community who
Hi all,
I'm coming around to this design now.
If we require users in some cluster (the root cluster) to be able to refer
to resources in virtual clusters for administration, and are not willing to
add protocol-specific support for this, it needs to be done through name
generation of some kind. I
This is fantastic news!
Thank you to everyone who contributed to 3.x to get ready for this
milestone, and everyone who worked on 4.0 to get it over the line.
Thanks David for running the release!
Greg
On Tue, Mar 18, 2025, 7:27 PM José Armando García Sancio
wrote:
> Awesome. Congratulations
[
https://issues.apache.org/jira/browse/KAFKA-17808?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-17808.
-
Resolution: Fixed
> InstanceAlreadyExistsException:
> kafka.admin.client:type=app-i
[
https://issues.apache.org/jira/browse/KAFKA-18863?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-18863.
-
Fix Version/s: 4.1.0
Resolution: Fixed
> Runtime additions for connector multiversion
oad among a different number of producers..could you
> suggest java code for this. This is an api calling from dB stored procedure
> and they can call any number of times this api with messages as payload.
>
> How to find downstream bottleneck (network, brokers, partitions).
>
> On
Hi Giri,
Since nobody with more experience has answered yet, let me give you my
amateur understanding of this error.
The TimeoutException will appear whenever the load generation (code calling
the Producer) runs faster than all downstream components (Producer,
Network, Brokers, etc) can handle.
R
pecial filtering or a custom class
> loader. The concern here is that any class on the class path (trusted or
> not) could be loaded which could intercept messages or corrupt messages by
> compressing using an unknown algorithm.
>
> -Original Message-
> From: Greg Harri
name topics
> > created through a VC, a cluster admin will have a hard time identifying
> > topics and their meanings. I personally think that having some "pointers"
> > in the name helps a lot in operations. My second point is about the scope
> > of the changes we
[
https://issues.apache.org/jira/browse/KAFKA-18833?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-18833.
-
Resolution: Invalid
> Infinite loop on authenticcation error with AWS MSK
Hi Olasoji,
Thanks for the KIP! I find it addresses many of the concerns I had with
KIP-984. Thank you for following up on that!
1. Since the Compression interface is internal currently, it may contain
some components that were not designed with pluggability in mind, such as
the static methods. A
Hi All,
Thanks for this amazing KIP! I'm glad that Kafka is getting more accessible
multi-tenancy features, beyond the primitives that are already available.
1. I think one concept for hierarchical VCs could be to make it so there is
a "root VC" corresponding to the physical cluster, and define t
aged differently inside the
> same broker?
> To clarify, there is no intention to replace tiered storage and the chunk
> meant to work with it. Please see "Automatic chunk deletion" section in the
> KIP.
>
> I hope these answer your concerns.
> Thanks again for your revie
Hi De Gao,
Thanks for the KIP!
I'd like to re-raise the concerns that David and Justine have made,
especially the alternative of Tiered Storage and the increase in (client)
metadata complexity.
I don't think that the KIP contains a satisfactory explanation of why this
change is worth it compared
[
https://issues.apache.org/jira/browse/KAFKA-12869?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-12869.
-
Resolution: Won't Fix
The specified version is out-of-support and there have been num
[
https://issues.apache.org/jira/browse/KAFKA-18386?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-18386.
-
Resolution: Won't Fix
> Mirror Maker2 Pod CrashLoopBackoff When one DC is pow
[
https://issues.apache.org/jira/browse/KAFKA-15001?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-15001.
-
Resolution: Fixed
> CVE vulnerabilities in Je
[
https://issues.apache.org/jira/browse/KAFKA-15496?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-15496.
-
Resolution: Duplicate
> Upgrade Jetty to version 11.0
[
https://issues.apache.org/jira/browse/KAFKA-12359?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-12359.
-
Fix Version/s: 4.0.0
(was: 4.1.0)
Assignee: (was: Dongjin
[
https://issues.apache.org/jira/browse/KAFKA-16326?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-16326.
-
Resolution: Fixed
> Kafka Connect unable to find javax dependency on Quarkus update to
[
https://issues.apache.org/jira/browse/KAFKA-13027?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-13027.
-
Fix Version/s: 4.0.0
Resolution: Duplicate
> Support for Jakarta EE 9.x to al
Hi Gaurav,
Thanks for the KIP!
I've noticed the lack of Throwable-based constructors in the past, so I'm
glad you're bringing this up for discussion. I think it makes sense to be
able to add more information relevant for debugging configuration issues
directly as a cause.
1. Could you discuss #a
[
https://issues.apache.org/jira/browse/KAFKA-17792?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-17792.
-
Resolution: Fixed
> header parsing ends up timing out and using large quantities of memory
Hi all,
I just backported
https://issues.apache.org/jira/projects/KAFKA/issues/KAFKA-17792 to 4.0.
It resolves an out-of-memory scenario for Connect.
Thanks!
Greg
On Mon, Jan 27, 2025 at 9:33 AM Justine Olshan
wrote:
> Hey folks,
> I've found what I believe to be a performance regression that
ect
> > users.
> >
> > understandable but I have then the feeling that the change to the data
> > model will be not so easy and fast with the risk to block the
> > expansion outside connect.
> > Maybe starting with the expansion and if there will be more t
[
https://issues.apache.org/jira/browse/KAFKA-18211?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-18211.
-
Assignee: Snehashis Pal
Resolution: Fixed
> ClassGraph scanning does not correctly f
Greg Harris created KAFKA-18460:
---
Summary: MetadataSchemaCheckerTool verify-evolution-git is broken
in Git worktrees
Key: KAFKA-18460
URL: https://issues.apache.org/jira/browse/KAFKA-18460
Project
Greg Harris created KAFKA-18461:
---
Summary:
QuorumControllerMetricsIntegrationTest.testFailingOverIncrementsNewActiveControllerCount
flaky failing with NPE
Key: KAFKA-18461
URL: https://issues.apache.org/jira
Hi Mario,
Thanks for the KIP! I think that using the same data model inside and
outside of connect is valuable, and is a good motivation for this effort.
However, I have seen many situations where the existing data model is
insufficient for Connect plugin developers.
* It's a closed type system w
[
https://issues.apache.org/jira/browse/KAFKA-18436?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-18436.
-
Resolution: Fixed
> Revert KIP-891 from 4.0 rele
Greg Harris created KAFKA-18436:
---
Summary: Revert KIP-891 from 4.0 release
Key: KAFKA-18436
URL: https://issues.apache.org/jira/browse/KAFKA-18436
Project: Kafka
Issue Type: Task
[
https://issues.apache.org/jira/browse/KAFKA-18419?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-18419.
-
Resolution: Fixed
> Accept plugin.version configurations for transforms and predica
Greg Harris created KAFKA-18419:
---
Summary: Accept plugin.version configurations for transforms and
predicates
Key: KAFKA-18419
URL: https://issues.apache.org/jira/browse/KAFKA-18419
Project: Kafka
e, if we only have v3.9.1 release and no more maintenance for
> > v3.9.x, why should we accept this "medium risk" decision?
> >
> > Thanks.
> > Luke
> >
> >
> > On Tue, Dec 17, 2024 at 1:06 AM Greg Harris >
> > wrote:
> >
> >
gt; Best,
> Chia-Ping
>
> On 2024/12/02 21:42:23 Swikar Patel wrote:
> > Java 23 is not LTS (Long Term Support) by Oracle
> > Do we still want to proceed?
> >
> > Thanks
> > Swikar
> >
> > > On Dec 2, 2024, at 1:07 PM, Greg Harris
> wrote:
&g
[
https://issues.apache.org/jira/browse/KAFKA-18215?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-18215.
-
Fix Version/s: 4.0.0
Assignee: Snehashis Pal
Resolution: Fixed
> Acc
Greg Harris created KAFKA-18215:
---
Summary: Accept plugin.version configurations for connector and
converters
Key: KAFKA-18215
URL: https://issues.apache.org/jira/browse/KAFKA-18215
Project: Kafka
[
https://issues.apache.org/jira/browse/KAFKA-18021?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-18021.
-
Fix Version/s: 4.0.0
Resolution: Fixed
> Disabled MirrorCheckpointConnector thr
[
https://issues.apache.org/jira/browse/KAFKA-15387?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-15387.
-
Resolution: Fixed
> Deprecate and remove Connect's redundant task configurations r
[
https://issues.apache.org/jira/browse/KAFKA-18182?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-18182.
-
Fix Version/s: 4.0.0
Resolution: Fixed
> Add VersionRange to Plugins
Greg Harris created KAFKA-18182:
---
Summary: Add VersionRange to Plugins and DelegatingClassLoader APIs
Key: KAFKA-18182
URL: https://issues.apache.org/jira/browse/KAFKA-18182
Project: Kafka
is our consensus here? Should we disregard our users, and prioritize
ourselves first?
Thanks,
Greg
On Thu, Nov 21, 2024 at 12:52 PM Greg Harris wrote:
> Hi Ismael,
>
> Thanks for your responses.
>
> > At the same time, where do we draw the line when it
> > comes t
https://github.com/trinodb/trino/issues/23498
On Thu, Nov 21, 2024 at 12:16 PM Ismael Juma wrote:
> Hi Greg,
>
> Comments below.
>
> On Thu, Nov 21, 2024 at 8:07 AM Greg Harris
> wrote:
>
> > > Greg, why can't we set the relevant system property
> > > a
3.9 as
> > an
> > > LTS release. For example, will we continue with versions like 3.9.1,
> > 3.9.2,
> > > ... 3.9.100?
> > >
> > > If yes, we should backport both KIP-1006 and support for future JDKs.
> > >
> > > If not, backp
we backport things into 3.9.x, it will slow adoption of 4.x and
> increase our maintenance burden over time.
>
> Just my $0.02
>
> Thanks!
> David A
>
> On Wed, Nov 20, 2024 at 12:22 PM Greg Harris >
> wrote:
>
> > Hi all,
> >
> > Now that 3.9.0 is
Hi all,
Now that 3.9.0 is released and 4.0.x is progressing, I'd like to understand
everyone's expectations about the 3.9.x branch, and ask for a specific
consensus on Java 23 support.
Some context that I think is relevant to the discussion:
* KIP-1006 [1] proposes a backwards-compatible strategy
Hi Divij,
Thanks for the KIP! I'm happy that best practices from applications using
Kafka are being reflected in sane defaults, in particular the linger.ms
change always seems to be the first step to tune a producer configuration.
And it's following in the footsteps of Nagle's algorithm too :)
I
Greg Harris created KAFKA-18021:
---
Summary: Disabled MirrorCheckpointConnector throws
RetriableException on task config generation
Key: KAFKA-18021
URL: https://issues.apache.org/jira/browse/KAFKA-18021
Hi TaiJuWu,
Thanks for the KIP!
Can you explain in the KIP about the behavior when the number of acks is
different for individual records? I think the current description using the
word "straightforward" does little to explain that, and may actually be
hiding some complexity.
For example, the se
Hi all,
FYI the Documentation Plan was added to the template in September 2023
around the time ~979 was opened. I was not able to find any contemporary
discussion thread, does anyone know of where/when we discussed this change?
Not to say that I think the change to the template should be undone, a
[
https://issues.apache.org/jira/browse/KAFKA-17755?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris reopened KAFKA-17755:
-
> AbstractPartitionAssignor can not enable RackAwareAssignment base on lead
> rac
Hi Anton,
Thanks for the KIP! I think that looking at internal APIs as inspiration
for new external APIs is a good idea, and I'm glad that you found an
interface close to the problem you're trying to solve.
What sort of implementation do you have in mind for this interface? What
use-case does thi
Greg Harris created KAFKA-17871:
---
Summary: Source task source offset reads can block herder task
cancellation
Key: KAFKA-17871
URL: https://issues.apache.org/jira/browse/KAFKA-17871
Project: Kafka
t ? Do they get updated ?
>
> Sorry for all the questions, I have been using Kafka for several years and
> want to dive deep into it a little bit. I have become more interested and
> ready to find out on my own. But still look forward to your thoughts on
> this if the questions above do
Hi Priyanka,
This is definitely an improvement for single-schema Connectors, and simple
to configure.
I think mutable or multiple schemas are fair to consider out-of-scope, as
they are currently served by the existing schema management tooling and
would complicate this design.
I'm also interested
Hi Xiang,
Thanks for your question! That sentence is a justification for why the
partitionLeaderEpoch field is not included in the CRC.
If you mutate fields which are included in a CRC, you need to recompute the
CRC value. See [1] for mutating the maxTimestamp. Compare that with [2] for
setting t
Greg Harris created KAFKA-17857:
---
Summary: Move AbstractResetIntegrationTest and subclasses to tools
Key: KAFKA-17857
URL: https://issues.apache.org/jira/browse/KAFKA-17857
Project: Kafka
>
> > > On the off chance that an existing plugin is designed to accept a
> > "version"
> > > property, could we either 1) keep passing that property to plugins
> > instead
> > > of stripping it, or 2) rename our new property to something like
>
Greg Harris created KAFKA-17800:
---
Summary: Forbid the use of System.exit and Runtime exit
Key: KAFKA-17800
URL: https://issues.apache.org/jira/browse/KAFKA-17800
Project: Kafka
Issue Type
Greg Harris created KAFKA-17799:
---
Summary: Forbid use of the default system locale
Key: KAFKA-17799
URL: https://issues.apache.org/jira/browse/KAFKA-17799
Project: Kafka
Issue Type
Greg Harris created KAFKA-17798:
---
Summary: Add forbidden-apis linting to gradle build
Key: KAFKA-17798
URL: https://issues.apache.org/jira/browse/KAFKA-17798
Project: Kafka
Issue Type
Hi Christo,
Thanks for the KIP!
I had some thoughts that are relevant to Kafka broker out-of-space
conditions, and I'll offer them here to see if they make sense to
incorporate into this KIP, or if someone suggested them before.
1. Rather than respond to an out-of-disk condition after it has hap
Hey Snehashis,
Thanks for the KIP! +1 (binding)
Greg
On Fri, Oct 4, 2024 at 10:14 PM Snehashis wrote:
> Hi everyone
>
> I would like to call a vote for KIP-891. Please take a moment to review the
> proposal and submit your vote. Special thanks to Greg who helped to expand
> this to make it muc
[
https://issues.apache.org/jira/browse/KAFKA-15862?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris reopened KAFKA-15862:
-
> Remove SecurityManager Support
> --
>
>
[
https://issues.apache.org/jira/browse/KAFKA-15862?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-15862.
-
Fix Version/s: 4.0.0
Resolution: Fixed
> Remove SecurityManager Supp
Greg Harris created KAFKA-17639:
---
Summary: Add Java 23 to CI build matrix
Key: KAFKA-17639
URL: https://issues.apache.org/jira/browse/KAFKA-17639
Project: Kafka
Issue Type: Task
Greg Harris created KAFKA-17640:
---
Summary: Document Java 23 support and include release note
Key: KAFKA-17640
URL: https://issues.apache.org/jira/browse/KAFKA-17640
Project: Kafka
Issue Type
Greg Harris created KAFKA-17638:
---
Summary: Add support for Java 23
Key: KAFKA-17638
URL: https://issues.apache.org/jira/browse/KAFKA-17638
Project: Kafka
Issue Type: Improvement
Affects
Greg Harris created KAFKA-17627:
---
Summary: ConfigProvider TTLs do not restart Tasks
Key: KAFKA-17627
URL: https://issues.apache.org/jira/browse/KAFKA-17627
Project: Kafka
Issue Type: Bug
Hi Colin,
It has been brought to my attention that Java 23 is now GA, and Kafka
clients, brokers, and connect are now experiencing
UnsupportedOperationExceptions by default due our use of the
deprecated-for-removal SecurityManager.
This only happens upon upgrading to Java 23, so this is not a regr
on this. Isn't this work part of KIP-1006? That
> > KIP hasn't even been approved (although it seems very likely to be so).
> > Certainly it isn't in 3.9. The PR is also kind of large with 16 files
> > changed. I think supporting new JDK versions that we haven't sup
ements as a hard
> requirement i.e. 3.8 instead of [3.8]. I have updated the KIP and added a
> line there highlighting this.
>
> Regards
> Snehashis
>
> On Wed, Aug 21, 2024 at 9:24 PM Greg Harris
> wrote:
>
> > Hey Snehashis,
> >
> > Thanks for your rep
Hey Murali,
Thanks for raising this.
I think it would make sense to take another look at the dependencies
between the different CRUD objects that Kafka exposes, and add some sanity
checks where they make sense and can be implemented safely.
I was just speaking to a user that had some confusion f
Hey everyone,
Bumping this topic to see if anyone has experience migrating Streams jobs
between clusters with Mirror Maker, and how you handled repartition topics.
Thanks!
Greg
On Mon, Aug 12, 2024 at 8:55 AM Greg Harris wrote:
> Hi all,
>
> We were recently working to set up a Mir
Congratulations!
On Fri, Sep 6, 2024 at 9:39 AM Muralidhar Basani
wrote:
> Congrats Josep. Well deserved.
>
> -Murali
>
> On Fri, 6 Sep 2024 at 18:34, Bill Bejeck wrote:
>
> > Congratulations Josep, well deserved!
> >
> > -Bill
> >
> > On Fri, Sep 6, 2024 at 12:30 PM Chia-Ping Tsai
> wrote:
>
[
https://issues.apache.org/jira/browse/KAFKA-12818?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-12818.
-
Resolution: Duplicate
> Memory leakage when kafka connect 2.7 uses directory config provi
t as is but making it simpler is also fine. Also for converter versions
> specified as part of the worker configs I believe we concluded that this
> step need not be fatal during worker startup if the required version is not
> found but LMK if otherwise.
>
> Regards
> Snehashis
&g
Hi Ran,
The config.action.reload, subscribe, unsubscribe, and ConfigChangeCallback
features are partially-implemented. As far as I can tell if you're using
only AK connect-runtime and config providers, none of it is functional.
If you have a config provider that implements TTLs, the runtime will
[
https://issues.apache.org/jira/browse/KAFKA-17369?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-17369.
-
Fix Version/s: 4.0.0
Resolution: Fixed
> Remove org.reflections from logging
Greg Harris created KAFKA-17369:
---
Summary: Remove org.reflections from logging and licenses
Key: KAFKA-17369
URL: https://issues.apache.org/jira/browse/KAFKA-17369
Project: Kafka
Issue Type
[
https://issues.apache.org/jira/browse/KAFKA-15203?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Greg Harris resolved KAFKA-15203.
-
Fix Version/s: 4.0.0
(was: 5.0.0)
Assignee: (was: Ganesh
ecated)
> already receive a warning message on startup."
> Maybe we can remove it or add more words at the end?
>
>
> Anyway, +1 (binding) from me.
>
> Thank you.
> Luke
>
> On Tue, Jul 9, 2024 at 2:26 AM Greg Harris
> wrote:
>
> > Hi all,
&g
e KIP (LMK if you want me to
> take that up).
>
> Regards
> Snehashis
>
> On Tue, Jul 2, 2024 at 12:08 AM Greg Harris
> wrote:
>
> > Hey Snehashis,
> >
> > Sorry for the late reply, and thanks for helping close out the
> discussion.
> >
> >
Hi all,
We were recently working to set up a MirrorMaker2 flow to migrate a Streams
job between an existing Kafka cluster and a new Kafka cluster. We noticed
the following behavior:
* Streams sets infinite retention on the repartition topic, and uses the
AdminClient deleteRecords call to implemen
Greg Harris created KAFKA-17242:
---
Summary: MirrorCheckpointTask prints spurious timeout for starting
checkpoint and offset sync stores
Key: KAFKA-17242
URL: https://issues.apache.org/jira/browse/KAFKA-17242
Greg Harris created KAFKA-17233:
---
Summary: MirrorCheckpointConnector should use batched
listConsumerGroupOffsets
Key: KAFKA-17233
URL: https://issues.apache.org/jira/browse/KAFKA-17233
Project: Kafka
Greg Harris created KAFKA-17232:
---
Summary: MirrorCheckpointConnector does not generate task configs
if initial consumer group load times out
Key: KAFKA-17232
URL: https://issues.apache.org/jira/browse/KAFKA-17232
sense to me,
> and we should not have a 3.9 release branch, and trunk should stay on
> 3.9-SNAPSHOT for the time being...
>
>
> -Matthias
>
> On 7/30/24 3:03 PM, Greg Harris wrote:
> > Hi all,
> >
> > I'd like to clarify my understanding of the path forw
t; Afshin
> > > >> Moazami, Ahmed Najiub, Ahmed Sobeh, Akhilesh Chaganti, Almog Gavra,
> > > Alok
> > > >> Thatikunta, Alyssa Huang, Anatoly Popov, Andras Katona, Andrew
> > > >> Schofield, Anna Sophie Blee-Goldman, Antoine Pourchet, Anton
> Agest
Hi all,
I'd like to clarify my understanding of the path forward, the one I voted
for in KIP-1012 and what I understood to be the consensus in the 3.8.0
release thread.
1. If KIP-853 is feature-complete before October, Kafka 3.9 can be released
ASAP with KIP-853. There will be no 3.10 release, an
Hi Patrik,
Thanks for the KIP!
Your motivation for this KIP is reasonable, because it is definitely
possible for the ".internal" suffix to collide with real topics. It would
have been nice if the original design included some mm2-specific namespace
like "mm2.internal" to lessen the likelihood of
Hi all,
I agree that we are not yet ready for breaking changes on trunk, so I
opened a PR to bump to 3.10.0-SNAPSHOT:
https://github.com/apache/kafka/pull/16732
When KIP-853 is feature complete, we can bump to 4.0.0-SNAPSHOT.
Thanks,
Greg
On Tue, Jul 30, 2024 at 10:01 AM Josep Prat
wrote:
> H
Greg Harris created KAFKA-17220:
---
Summary: Define new metrics for MirrorMaker2
Key: KAFKA-17220
URL: https://issues.apache.org/jira/browse/KAFKA-17220
Project: Kafka
Issue Type: New Feature
Greg Harris created KAFKA-17207:
---
Summary: ConnectWorkerIntegrationTest.testRequestTimeouts times
out in stop(), leaking clients
Key: KAFKA-17207
URL: https://issues.apache.org/jira/browse/KAFKA-17207
Greg Harris created KAFKA-17204:
---
Summary: KafkaStreamsCloseOptionsIntegrationTest.before leaks
AdminClient
Key: KAFKA-17204
URL: https://issues.apache.org/jira/browse/KAFKA-17204
Project: Kafka
Greg Harris created KAFKA-17203:
---
Summary: StreamThread leaking producer instances
Key: KAFKA-17203
URL: https://issues.apache.org/jira/browse/KAFKA-17203
Project: Kafka
Issue Type: Test
1 - 100 of 485 matches
Mail list logo