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 effectiveness.
>
> +1 (binding)
>
>
> ~Satish.
>
> On Wed, 3 Jan 2024 at 06:59, Justine Olshan
>  wrote:
> >
> > Thanks for driving this.
> >
> > +1 (binding) from me.
> >
> > Justine
> >
> > On Tue, Jan 2, 2024 at 4:30 PM Ismael Juma  wrote:
> >
> > > Hi all,
> > >
> > > I would like to start a vote on KIP-1013.
> > >
> > > As stated in the discussion thread, this KIP was proposed after the KIP
> > > freeze for Apache Kafka 3.7, but it is purely a documentation update
> (if we
> > > decide to adopt it) and I believe it would serve our users best if we
> > > communicate the deprecation for removal sooner (i.e. 3.7) rather than
> later
> > > (i.e. 3.8).
> > >
> > > Please take a look and cast your vote.
> > >
> > > Link:
> > >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=284789510
> > >
> > > Ismael
> > >
>


[jira] [Resolved] (KAFKA-16034) AsyncKafkaConsumer will get Invalid Request error when trying to rejoin on fenced/unknown member Id

2024-01-02 Thread Phuc Hong Tran (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-16034?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Phuc Hong Tran resolved KAFKA-16034.

Resolution: Fixed

> AsyncKafkaConsumer will get Invalid Request error when trying to rejoin on 
> fenced/unknown member Id
> ---
>
> Key: KAFKA-16034
> URL: https://issues.apache.org/jira/browse/KAFKA-16034
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Philip Nee
>Assignee: Phuc Hong Tran
>Priority: Major
> Fix For: 3.8.0
>
>
> The consumer will log invalid request error when joining from fenced/unknown 
> member id because we didn't reset the HeartbeatState and we won't send the 
> needed fields (rebalanceTimeoutMs for example) when joining.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #2531

2024-01-02 Thread Apache Jenkins Server
See 




Jenkins build is unstable: Kafka » Kafka Branch Builder » 3.7 #45

2024-01-02 Thread Apache Jenkins Server
See 




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

2024-01-02 Thread Satish Duggana
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 effectiveness.

+1 (binding)


~Satish.

On Wed, 3 Jan 2024 at 06:59, Justine Olshan
 wrote:
>
> Thanks for driving this.
>
> +1 (binding) from me.
>
> Justine
>
> On Tue, Jan 2, 2024 at 4:30 PM Ismael Juma  wrote:
>
> > Hi all,
> >
> > I would like to start a vote on KIP-1013.
> >
> > As stated in the discussion thread, this KIP was proposed after the KIP
> > freeze for Apache Kafka 3.7, but it is purely a documentation update (if we
> > decide to adopt it) and I believe it would serve our users best if we
> > communicate the deprecation for removal sooner (i.e. 3.7) rather than later
> > (i.e. 3.8).
> >
> > Please take a look and cast your vote.
> >
> > Link:
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=284789510
> >
> > Ismael
> >


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

2024-01-02 Thread Justine Olshan
Thanks for driving this.

+1 (binding) from me.

Justine

On Tue, Jan 2, 2024 at 4:30 PM Ismael Juma  wrote:

> Hi all,
>
> I would like to start a vote on KIP-1013.
>
> As stated in the discussion thread, this KIP was proposed after the KIP
> freeze for Apache Kafka 3.7, but it is purely a documentation update (if we
> decide to adopt it) and I believe it would serve our users best if we
> communicate the deprecation for removal sooner (i.e. 3.7) rather than later
> (i.e. 3.8).
>
> Please take a look and cast your vote.
>
> Link:
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=284789510
>
> Ismael
>


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

2024-01-02 Thread Ismael Juma
Hi all,

I would like to start a vote on KIP-1013.

As stated in the discussion thread, this KIP was proposed after the KIP
freeze for Apache Kafka 3.7, but it is purely a documentation update (if we
decide to adopt it) and I believe it would serve our users best if we
communicate the deprecation for removal sooner (i.e. 3.7) rather than later
(i.e. 3.8).

Please take a look and cast your vote.

Link:
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=284789510

Ismael


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

2024-01-02 Thread Ismael Juma
It seems like people are generally in favor, I'll start a vote thread.

Ismael

On Fri, Dec 22, 2023 at 10:39 AM Ismael Juma  wrote:

> Hi all,
>
> I was watching the Java Highlights of 2023 from Nicolai Parlog[1] and it
> became clear that many projects are moving to Java 17 for its developer
> productivity improvements. It occurred to me that there is also an
> opportunity for the Apache Kafka project and I wrote a quick KIP with the
> proposal. Please take a look and let me know what you think:
>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=284789510
>
> P.S. I am aware that we're past the KIP freeze for Apache Kafka 3.7, but
> the proposed change would only change documentation and it's strictly
> better to share this information in 3.7 than 3.8 (if we decide to do it).
>
> [1] https://youtu.be/NxpHg_GzpnY?si=wA57g9kAhYulrlUO=411
>


Re: [Question] About Kafka producer design decision making

2024-01-02 Thread Ismael Juma
I should also clarify that Chia-Ping  took it over from me originally and
improved the proposal significantly. I think he also got busy and hasn't
been able to spend time on it for a while though.

Ismael

On Tue, Jan 2, 2024 at 4:08 PM Ismael Juma  wrote:

> I had written a KIP a while back:
>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=100829459
>
> Happy for someone to pick it up and drive it forward.
>
> Ismael
>
> On Tue, Jan 2, 2024 at 2:00 PM Justine Olshan 
> wrote:
>
>> Hey folks --
>>  I think this is a good conversation. Given we plan to drop support for
>> Java 8 in 4.0 this seems like a good time to consider this change. Perhaps
>> we should file a JIRA ticket and maybe create a KIP to discuss?
>>
>> One thing we should consider however, is if we want some of these
>> operations to be asynchronous. I know I got into some tricky areas of
>> Kafka
>> recently that assumed operations completed in a given order so it is good
>> to confirm such changes are safe.
>>
>> Looking forward to further discussion,
>> Justine
>>
>> On Tue, Jan 2, 2024 at 1:14 PM Philip Nee  wrote:
>>
>> > hey sean - a lot of uses of the Futures are in the public API and
>> therefore
>> > take voting/effort to be changed.  i don't know any reason for
>> > intentionally avoiding the use of CompletableFuture, however, others
>> might
>> > know more than I do.
>> >
>> > thanks,
>> > P
>> >
>> > On Tue, Nov 14, 2023 at 1:27 AM 신수웅(Sean Sin) 
>> > wrote:
>> >
>> > > Dear Apache Kakfa Developers,
>> > >
>> > > I'm 4-year SWE in South Korea.
>> > > I have some questions while watching Kafka Producer API.
>> > >
>> > > *Why Use "Future" and Not "CompletableFuture"?*
>> > >
>> > > In the case of "Future", blocking occurs when calling "*get()*", so I
>> > > thought "Computable Future" would be better when doing more
>> asynchronous
>> > > operations.
>> > >
>> > > I looked at the Java API document
>> > > <
>> > >
>> >
>> https://kafka.apache.org/36/javadoc/org/apache/kafka/common/KafkaFuture.html#thenApply(org.apache.kafka.common.KafkaFuture.BaseFunction)
>> > > >
>> > > based on the latest version, version 3.6.x.
>> > >
>> > > If you look at that version, you can see that the Future object
>> provides
>> > > the "toCompletionStage() "method, which can convert "KafkaFuture" to
>> > > "ComputableFuture".
>> > >
>> > > In response to this, I think that in the initial design decision
>> process,
>> > > we considered compatibility issues under JDK 1.8 and the level of
>> > knowledge
>> > > of the learning curve or developer when introducing ComputableFuture,
>> > but I
>> > > wonder if this is correct.
>> > >
>> > > In addition, I wonder if it is recommended to use the
>> > "toCompletionStage()"
>> > > method to produce more non-blocking if we assume JDK 1.8 or higher.
>> > >
>> > > Thanks.
>> > > Su-Ung Shin.
>> > >
>> >
>>
>


Re: [Question] About Kafka producer design decision making

2024-01-02 Thread Ismael Juma
I had written a KIP a while back:

https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=100829459

Happy for someone to pick it up and drive it forward.

Ismael

On Tue, Jan 2, 2024 at 2:00 PM Justine Olshan 
wrote:

> Hey folks --
>  I think this is a good conversation. Given we plan to drop support for
> Java 8 in 4.0 this seems like a good time to consider this change. Perhaps
> we should file a JIRA ticket and maybe create a KIP to discuss?
>
> One thing we should consider however, is if we want some of these
> operations to be asynchronous. I know I got into some tricky areas of Kafka
> recently that assumed operations completed in a given order so it is good
> to confirm such changes are safe.
>
> Looking forward to further discussion,
> Justine
>
> On Tue, Jan 2, 2024 at 1:14 PM Philip Nee  wrote:
>
> > hey sean - a lot of uses of the Futures are in the public API and
> therefore
> > take voting/effort to be changed.  i don't know any reason for
> > intentionally avoiding the use of CompletableFuture, however, others
> might
> > know more than I do.
> >
> > thanks,
> > P
> >
> > On Tue, Nov 14, 2023 at 1:27 AM 신수웅(Sean Sin) 
> > wrote:
> >
> > > Dear Apache Kakfa Developers,
> > >
> > > I'm 4-year SWE in South Korea.
> > > I have some questions while watching Kafka Producer API.
> > >
> > > *Why Use "Future" and Not "CompletableFuture"?*
> > >
> > > In the case of "Future", blocking occurs when calling "*get()*", so I
> > > thought "Computable Future" would be better when doing more
> asynchronous
> > > operations.
> > >
> > > I looked at the Java API document
> > > <
> > >
> >
> https://kafka.apache.org/36/javadoc/org/apache/kafka/common/KafkaFuture.html#thenApply(org.apache.kafka.common.KafkaFuture.BaseFunction)
> > > >
> > > based on the latest version, version 3.6.x.
> > >
> > > If you look at that version, you can see that the Future object
> provides
> > > the "toCompletionStage() "method, which can convert "KafkaFuture" to
> > > "ComputableFuture".
> > >
> > > In response to this, I think that in the initial design decision
> process,
> > > we considered compatibility issues under JDK 1.8 and the level of
> > knowledge
> > > of the learning curve or developer when introducing ComputableFuture,
> > but I
> > > wonder if this is correct.
> > >
> > > In addition, I wonder if it is recommended to use the
> > "toCompletionStage()"
> > > method to produce more non-blocking if we assume JDK 1.8 or higher.
> > >
> > > Thanks.
> > > Su-Ung Shin.
> > >
> >
>


Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #2530

2024-01-02 Thread Apache Jenkins Server
See 




[jira] [Created] (KAFKA-16076) RestClient Interrupting the thread in case of InterruptedException

2024-01-02 Thread Mehrdad Karami (Jira)
Mehrdad Karami created KAFKA-16076:
--

 Summary: RestClient Interrupting the thread in case of 
InterruptedException
 Key: KAFKA-16076
 URL: https://issues.apache.org/jira/browse/KAFKA-16076
 Project: Kafka
  Issue Type: Bug
  Components: KafkaConnect
Reporter: Mehrdad Karami


In RestClient class, `httpRequest` is being called with different threads. An 
`InterruptedException` in case of failure is used to handle its specific 
exceptions, nevertheless it's forgot to call 
`Thread.currentThread().interrupt()`.

In general, it's a good practice to call this so the rest of code know the 
thread was interrupted already.
Note:
Some methods that cause a thread to wait or sleep (like 
`{{{}Thread.sleep()`{}}}) will check this flag. If they see it’s set, they’ll 
stop waiting/sleeping and throw an `{{{}InterruptedException`{}}}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Build failed in Jenkins: Kafka » Kafka Branch Builder » 3.7 #44

2024-01-02 Thread Apache Jenkins Server
See 


Changes:


--
[...truncated 453933 lines...]
Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldClearTaskTimeoutOnProcessed() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldUnassignTaskWhenRequired() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldUnassignTaskWhenRequired() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldClearTaskReleaseFutureOnShutdown() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldClearTaskReleaseFutureOnShutdown() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldProcessTasks() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldProcessTasks() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldPunctuateStreamTime() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldPunctuateStreamTime() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldShutdownTaskExecutor() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldShutdownTaskExecutor() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldAwaitProcessableTasksIfNoneAssignable() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldAwaitProcessableTasksIfNoneAssignable() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > 
shouldRespectPunctuationDisabledByTaskExecutionMetadata() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > 
shouldRespectPunctuationDisabledByTaskExecutionMetadata() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldSetTaskTimeoutOnTimeoutException() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldSetTaskTimeoutOnTimeoutException() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldPunctuateSystemTime() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldPunctuateSystemTime() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldUnassignTaskWhenNotProgressing() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldUnassignTaskWhenNotProgressing() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldNotFlushOnException() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > shouldNotFlushOnException() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > 
shouldRespectProcessingDisabledByTaskExecutionMetadata() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskExecutorTest > 
shouldRespectProcessingDisabledByTaskExecutionMetadata() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskManagerTest > shouldLockAnEmptySetOfTasks() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskManagerTest > shouldLockAnEmptySetOfTasks() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskManagerTest > shouldAssignTasksThatCanBeSystemTimePunctuated() 
STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskManagerTest > shouldAssignTasksThatCanBeSystemTimePunctuated() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskManagerTest > shouldNotUnassignNotOwnedTask() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskManagerTest > shouldNotUnassignNotOwnedTask() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskManagerTest > shouldNotSetUncaughtExceptionsTwice() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskManagerTest > shouldNotSetUncaughtExceptionsTwice() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskManagerTest > shouldReturnFromAwaitOnAdding() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskManagerTest > shouldReturnFromAwaitOnAdding() PASSED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskManagerTest > shouldShutdownTaskExecutors() STARTED

Gradle Test Run :streams:test > Gradle Test Executor 90 > 
DefaultTaskManagerTest > shouldShutdownTaskExecutors() PASSED

Gradle Test Run 

Re: [Question] About Kafka producer design decision making

2024-01-02 Thread Justine Olshan
Hey folks --
 I think this is a good conversation. Given we plan to drop support for
Java 8 in 4.0 this seems like a good time to consider this change. Perhaps
we should file a JIRA ticket and maybe create a KIP to discuss?

One thing we should consider however, is if we want some of these
operations to be asynchronous. I know I got into some tricky areas of Kafka
recently that assumed operations completed in a given order so it is good
to confirm such changes are safe.

Looking forward to further discussion,
Justine

On Tue, Jan 2, 2024 at 1:14 PM Philip Nee  wrote:

> hey sean - a lot of uses of the Futures are in the public API and therefore
> take voting/effort to be changed.  i don't know any reason for
> intentionally avoiding the use of CompletableFuture, however, others might
> know more than I do.
>
> thanks,
> P
>
> On Tue, Nov 14, 2023 at 1:27 AM 신수웅(Sean Sin) 
> wrote:
>
> > Dear Apache Kakfa Developers,
> >
> > I'm 4-year SWE in South Korea.
> > I have some questions while watching Kafka Producer API.
> >
> > *Why Use "Future" and Not "CompletableFuture"?*
> >
> > In the case of "Future", blocking occurs when calling "*get()*", so I
> > thought "Computable Future" would be better when doing more asynchronous
> > operations.
> >
> > I looked at the Java API document
> > <
> >
> https://kafka.apache.org/36/javadoc/org/apache/kafka/common/KafkaFuture.html#thenApply(org.apache.kafka.common.KafkaFuture.BaseFunction)
> > >
> > based on the latest version, version 3.6.x.
> >
> > If you look at that version, you can see that the Future object provides
> > the "toCompletionStage() "method, which can convert "KafkaFuture" to
> > "ComputableFuture".
> >
> > In response to this, I think that in the initial design decision process,
> > we considered compatibility issues under JDK 1.8 and the level of
> knowledge
> > of the learning curve or developer when introducing ComputableFuture,
> but I
> > wonder if this is correct.
> >
> > In addition, I wonder if it is recommended to use the
> "toCompletionStage()"
> > method to produce more non-blocking if we assume JDK 1.8 or higher.
> >
> > Thanks.
> > Su-Ung Shin.
> >
>


Re: [Question] About Kafka producer design decision making

2024-01-02 Thread Philip Nee
hey sean - a lot of uses of the Futures are in the public API and therefore
take voting/effort to be changed.  i don't know any reason for
intentionally avoiding the use of CompletableFuture, however, others might
know more than I do.

thanks,
P

On Tue, Nov 14, 2023 at 1:27 AM 신수웅(Sean Sin)  wrote:

> Dear Apache Kakfa Developers,
>
> I'm 4-year SWE in South Korea.
> I have some questions while watching Kafka Producer API.
>
> *Why Use "Future" and Not "CompletableFuture"?*
>
> In the case of "Future", blocking occurs when calling "*get()*", so I
> thought "Computable Future" would be better when doing more asynchronous
> operations.
>
> I looked at the Java API document
> <
> https://kafka.apache.org/36/javadoc/org/apache/kafka/common/KafkaFuture.html#thenApply(org.apache.kafka.common.KafkaFuture.BaseFunction)
> >
> based on the latest version, version 3.6.x.
>
> If you look at that version, you can see that the Future object provides
> the "toCompletionStage() "method, which can convert "KafkaFuture" to
> "ComputableFuture".
>
> In response to this, I think that in the initial design decision process,
> we considered compatibility issues under JDK 1.8 and the level of knowledge
> of the learning curve or developer when introducing ComputableFuture, but I
> wonder if this is correct.
>
> In addition, I wonder if it is recommended to use the "toCompletionStage()"
> method to produce more non-blocking if we assume JDK 1.8 or higher.
>
> Thanks.
> Su-Ung Shin.
>


Re: [DISCUSS] Kafka distributed transaction support

2024-01-02 Thread Justine Olshan
I believe Artem also had some conversations offline about XA.

If I recall correctly, he didn't plan to include it in KIP-939 but was
happy to leave room for potential KIPs in the future.
Please feel free to continue the conversation on the thread. :)

Justine

On Tue, Jan 2, 2024 at 12:05 PM Greg Harris 
wrote:

> Hi Rowland,
>
> First of all, welcome to the community, and thanks for thinking about
> the future of Kafka!
>
> I'm not very familiar with X/Open XA, but from the documentation I
> read, it appears most related to KIP-939: Support Participation in 2PC
> [1] currently in-progress. You may be interested in contributing to
> the discussion [2] for that KIP to ensure that it is easy to use
> within an XA context. I see that someone else in that thread has
> mentioned XA, but no conclusions appear to have been reached. I'm sure
> that Artem would be interested to hear your use-case and vision for
> using Kafka with XA.
>
> [1]:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-939%3A+Support+Participation+in+2PC
> [2]: https://lists.apache.org/thread/wbs9sqs3z1tdm7ptw5j4o9osmx9s41nf
>
> Thanks!
> Greg
>
>
> On Mon, Jan 1, 2024 at 2:20 PM Rowland Smith  wrote:
> >
> > Hi All,
> >
> > I am new to the Kafka developer community. After learning more about
> > Kafka's transactional capabilities recently, I became interested in
> > exploring what would be required to provide support for the XA interface
> > specified in the X/ Open Distributed Processing Model in the Kafka
> producer
> > client. I have put together a proof of concept to satisfy my curiosity,
> and
> > based on that work, I think that extending the Kafka producer with XA
> > support is doable with reasonable effort.
> >
> > As I understand the Kafka development team's process, the first step in
> the
> > process would be to produce a KIP describing the feature's goals and
> > design. My question in this email is whether XA support has ever been
> > considered previously by the PMC and if so, with what result. I don't
> want
> > to spend time working on a KIP if XA support is not something that the
> PMC
> > sees value in including and supporting in the Kafka codebase.
> >
> > Any feedback would be appreciated. I am excited to work on this feature
> if
> > there is interest in the community.
> >
> > Regards,
> > Rowland
>


[jira] [Created] (KAFKA-16075) TLS configuration not validated in KRaft controller-only nodes

2024-01-02 Thread Jakub Scholz (Jira)
Jakub Scholz created KAFKA-16075:


 Summary: TLS configuration not validated in KRaft controller-only 
nodes
 Key: KAFKA-16075
 URL: https://issues.apache.org/jira/browse/KAFKA-16075
 Project: Kafka
  Issue Type: Bug
  Components: kraft
Affects Versions: 3.6.1
Reporter: Jakub Scholz


When the Kafka broker node (either a broker in ZooKeeper based cluster or node 
with a broker role in a KRaft cluster) has an incorrect TLS configuration such 
as unsupported TLS cipher suite, it seems to throw a {{ConfigException}} and 
shutdown:
{code:java}
2024-01-02 13:50:24,895 ERROR Exiting Kafka due to fatal exception during 
startup. (kafka.Kafka$) [main]
org.apache.kafka.common.config.ConfigException: Invalid value 
java.lang.IllegalArgumentException: Unsupported CipherSuite: 
TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305 for configuration A client SSLEngine 
created with the provided settings can't connect to a server SSLEngine created 
with those settings.
at 
org.apache.kafka.common.security.ssl.SslFactory.configure(SslFactory.java:102)
at 
org.apache.kafka.common.network.SslChannelBuilder.configure(SslChannelBuilder.java:73)
at 
org.apache.kafka.common.network.ChannelBuilders.create(ChannelBuilders.java:192)
at 
org.apache.kafka.common.network.ChannelBuilders.serverChannelBuilder(ChannelBuilders.java:107)
at kafka.network.Processor.(SocketServer.scala:973)
at kafka.network.Acceptor.newProcessor(SocketServer.scala:879)
at 
kafka.network.Acceptor.$anonfun$addProcessors$1(SocketServer.scala:849)
at scala.collection.immutable.Range.foreach$mVc$sp(Range.scala:190)
at kafka.network.Acceptor.addProcessors(SocketServer.scala:848)
at kafka.network.DataPlaneAcceptor.configure(SocketServer.scala:523)
at 
kafka.network.SocketServer.createDataPlaneAcceptorAndProcessors(SocketServer.scala:251)
at kafka.network.SocketServer.$anonfun$new$31(SocketServer.scala:175)
at 
kafka.network.SocketServer.$anonfun$new$31$adapted(SocketServer.scala:175)
at scala.collection.IterableOnceOps.foreach(IterableOnce.scala:576)
at scala.collection.IterableOnceOps.foreach$(IterableOnce.scala:574)
at scala.collection.AbstractIterable.foreach(Iterable.scala:933)
at kafka.network.SocketServer.(SocketServer.scala:175)
at kafka.server.BrokerServer.startup(BrokerServer.scala:242)
at 
kafka.server.KafkaRaftServer.$anonfun$startup$2(KafkaRaftServer.scala:96)
at 
kafka.server.KafkaRaftServer.$anonfun$startup$2$adapted(KafkaRaftServer.scala:96)
at scala.Option.foreach(Option.scala:437)
at kafka.server.KafkaRaftServer.startup(KafkaRaftServer.scala:96)
at kafka.Kafka$.main(Kafka.scala:113)
at kafka.Kafka.main(Kafka.scala) {code}
But in a KRaft controller-only nodes, such validation does not seem to happen 
and the broker keeps running and looping with this warning:
{code:java}
2024-01-02 13:53:10,186 WARN [RaftManager id=1] Error connecting to node 
my-cluster-controllers-0.my-cluster-kafka-brokers.myproject.svc.cluster.local:9090
 (id: 0 rack: null) (org.apache.kafka.clients.NetworkClient) 
[kafka-1-raft-outbound-request-thread]
java.io.IOException: Channel could not be created for socket 
java.nio.channels.SocketChannel[closed]
at 
org.apache.kafka.common.network.Selector.buildAndAttachKafkaChannel(Selector.java:348)
at 
org.apache.kafka.common.network.Selector.registerChannel(Selector.java:329)
at org.apache.kafka.common.network.Selector.connect(Selector.java:256)
at 
org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:1032)
at org.apache.kafka.clients.NetworkClient.ready(NetworkClient.java:301)
at 
org.apache.kafka.server.util.InterBrokerSendThread.sendRequests(InterBrokerSendThread.java:145)
at 
org.apache.kafka.server.util.InterBrokerSendThread.pollOnce(InterBrokerSendThread.java:108)
at 
org.apache.kafka.server.util.InterBrokerSendThread.doWork(InterBrokerSendThread.java:136)
at 
org.apache.kafka.server.util.ShutdownableThread.run(ShutdownableThread.java:130)
Caused by: org.apache.kafka.common.KafkaException: 
java.lang.IllegalArgumentException: Unsupported CipherSuite: 
TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305
at 
org.apache.kafka.common.network.SslChannelBuilder.buildChannel(SslChannelBuilder.java:111)
at 
org.apache.kafka.common.network.Selector.buildAndAttachKafkaChannel(Selector.java:338)
... 8 more
Caused by: java.lang.IllegalArgumentException: Unsupported CipherSuite: 
TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305
at 
java.base/sun.security.ssl.CipherSuite.validValuesOf(CipherSuite.java:978)
at 
java.base/sun.security.ssl.SSLEngineImpl.setEnabledCipherSuites(SSLEngineImpl.java:864)
at 

Re: [DISCUSS] Kafka distributed transaction support

2024-01-02 Thread Greg Harris
Hi Rowland,

First of all, welcome to the community, and thanks for thinking about
the future of Kafka!

I'm not very familiar with X/Open XA, but from the documentation I
read, it appears most related to KIP-939: Support Participation in 2PC
[1] currently in-progress. You may be interested in contributing to
the discussion [2] for that KIP to ensure that it is easy to use
within an XA context. I see that someone else in that thread has
mentioned XA, but no conclusions appear to have been reached. I'm sure
that Artem would be interested to hear your use-case and vision for
using Kafka with XA.

[1]: 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-939%3A+Support+Participation+in+2PC
[2]: https://lists.apache.org/thread/wbs9sqs3z1tdm7ptw5j4o9osmx9s41nf

Thanks!
Greg


On Mon, Jan 1, 2024 at 2:20 PM Rowland Smith  wrote:
>
> Hi All,
>
> I am new to the Kafka developer community. After learning more about
> Kafka's transactional capabilities recently, I became interested in
> exploring what would be required to provide support for the XA interface
> specified in the X/ Open Distributed Processing Model in the Kafka producer
> client. I have put together a proof of concept to satisfy my curiosity, and
> based on that work, I think that extending the Kafka producer with XA
> support is doable with reasonable effort.
>
> As I understand the Kafka development team's process, the first step in the
> process would be to produce a KIP describing the feature's goals and
> design. My question in this email is whether XA support has ever been
> considered previously by the PMC and if so, with what result. I don't want
> to spend time working on a KIP if XA support is not something that the PMC
> sees value in including and supporting in the Kafka codebase.
>
> Any feedback would be appreciated. I am excited to work on this feature if
> there is interest in the community.
>
> Regards,
> Rowland


Re: [VOTE] KIP-1004: Enforce tasks.max property in Kafka Connect

2024-01-02 Thread Hector Geraldino (BLOOMBERG/ 919 3RD A)
+1 (non-binding)

Thanks Chris!

From: dev@kafka.apache.org At: 01/02/24 11:49:18 UTC-5:00To:  
dev@kafka.apache.org
Subject: Re: [VOTE] KIP-1004: Enforce tasks.max property in Kafka Connect

Hi all,

Happy New Year! Wanted to give this a bump now that the holidays are over
for a lot of us. Looking forward to people's thoughts!

Cheers,

Chris

On Mon, Dec 4, 2023 at 10:36 AM Chris Egerton  wrote:

> Hi all,
>
> I'd like to call for a vote on KIP-1004, which adds enforcement for the
> tasks.max connector property in Kafka Connect.
>
> The KIP:
> 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1004%3A+Enforce+tasks.max+
property+in+Kafka+Connect
>
> The discussion thread:
> https://lists.apache.org/thread/scx75cjwm19jyt19wxky41q9smf5nx6d
>
> Cheers,
>
> Chris
>




Re: [DISCUSS] KIP-891: Running multiple versions of a connector.

2024-01-02 Thread Greg Harris
Hi Snehashis,

Thank you for the KIP! This is something I've wanted for a long time.

I know the discussion has gone cold, are you still interested in
pursuing this feature? I'll make time to review the KIP if you are
still accepting comments.

Thanks,
Greg

On Tue, Nov 22, 2022 at 12:29 PM Snehashis  wrote:
>
> Thanks for the points Sagar.
>
> > 1) Should we update the GET /connectors endpoint to include the version of
> > the plugin that is running? It could be useful to figure out the version
> of
> > the plugin or I am assuming it gets returned by the expand=info call?
>
> I think this is good to have and possible future enhancement. The version
> info will be present in the config of the connector if the user has
> specified the version. Otherwise it is the latest version which the user
> can find out from the connector-plugin endpoint. The information can be
> introduced to the response of the GET /connectors endpoint itself, however
> the most ideal way of doing this would be to get the currently running
> instance of the connector and get the version directly from there. This is
> slightly tricky as the connector could be running in a different node.
> One way to do this would be to persist the version information in the
> status backing store during instantiation of the connector. It requires
> some more thought and since the version is part of the configs if provided
> and evident otherwise, I have not included it in this KIP.
>
> > 2) I am not aware of this and hence asking, can 2 connectors with
> different
> > versions have the same name? Does the plugin isolation allow this? This
> > could have a bearing when using the lifecycle endpoints for connectors
> like
> > DELETE etc.
>
> All connectors in a cluster need to have uniquire connector names
> regardless of what version of the plugin the connector is running
> underneath. This is something enforced by the connect runtime itself. All
> connect CRUD operations are keyed on the connector name so there will not
> be an issue.
>
> Regards
> Snehashis
>
> On Tue, Nov 22, 2022 at 3:16 PM Sagar  wrote:
>
> > Hey Snehashsih,
> >
> > Thanks for the KIP. It looks like a very useful feature. Couple of
> > small-ish points, let me know what you think:
> >
> > 1) Should we update the GET /connectors endpoint to include the version of
> > the plugin that is running? It could be useful to figure out the version of
> > the plugin or I am assuming it gets returned by the expand=info call?
> > 2) I am not aware of this and hence asking, can 2 connectors with different
> > versions have the same name? Does the plugin isolation allow this? This
> > could have a bearing when using the lifecycle endpoints for connectors like
> > DELETE etc.
> >
> > Thanks!
> > Sagar.
> >
> >
> > On Tue, Nov 22, 2022 at 2:10 PM Ashwin 
> > wrote:
> >
> > > Hi Snehasis,
> > >
> > > > IIUC (please correct me if I am wrong here), what you highlighted
> > above,
> > > is
> > > a versioning scheme for a connector config for the same connector (and
> > not
> > > different versions of a connector plugin).
> > >
> > > Sorry for not being more precise in my wording -  I meant registering
> > > versions of schema for connector config.
> > >
> > > Let's take the example of a fictional connector which uses a fictional
> > AWS
> > > service.
> > >
> > > Fictional Connector Config schema version:2.0
> > > ---
> > > {
> > >   "$schema": "http://json-schema.org/draft-04/schema#;,
> > >   "type": "object",
> > >   "properties": {
> > > "name": {
> > >   "type": "string"
> > > },
> > > "schema_version": {
> > >   "type": "string"
> > > },
> > > "aws_access_key": {
> > >   "type": "string"
> > > },
> > > "aws_secret_key": {
> > >   "type": "string"
> > > }
> > >   },
> > >   "required": [
> > > "name",
> > > "schema_version",
> > > "aws_access_key",
> > > "aws_secret_key"
> > >   ]
> > > }
> > >
> > > Fictional Connector config schema version:3.0
> > > ---
> > > {
> > >   "$schema": "http://json-schema.org/draft-04/schema#;,
> > >   "type": "object",
> > >   "properties": {
> > > "name": {
> > >   "type": "string"
> > > },
> > > "schema_version": {
> > >   "type": "string"
> > > },
> > > "iam_role": {
> > >   "type": "string"
> > > }
> > >   },
> > >   "required": [
> > > "name",
> > > "schema_version",
> > > "iam_role"
> > >   ]
> > > }
> > >
> > > The connector which supports Fictional config schema 2.0  will validate
> > the
> > > access key and secret key.
> > > Whereas a connector which supports config with schema version 3.0 will
> > only
> > > validate the IAM role.
> > >
> > > This is the alternative which I wanted to suggest. Each plugin will
> > > register the schema versions of connector config which it supports.
> > >
> > > The plugin paths may be optionally different i.e  we don't have to
> > > mandatorily add a new plugin path to support a new schema version.
> > >
> > > 

Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #2529

2024-01-02 Thread Apache Jenkins Server
See 




Re: Apache Kafka 3.7.0 Release

2024-01-02 Thread Almog Gavra
Hello Stan,

I wanted to give you a heads up that
https://github.com/apache/kafka/pull/15073 (
https://issues.apache.org/jira/browse/KAFKA-16046) was identified as a
blocker regression and should be merged to trunk by EOD.

Cheers,
Almog

On Tue, Jan 2, 2024 at 4:20 AM Stanislav Kozlovski
 wrote:

> Hi Apoorv,
>
> Thanks for taking ownership and looking into this! One more caveat is that
> I believe this first publish is ran with JDK 8, as the release.py runs with
> both JDK 8 and (if I recall correctly) 17 versions. This seems to fail on
> the first one - so JDK 8.
> Not sure if that is related in any way. And I'm also not sure if it should
> be kafka-clients or just clients.
>
> On Sat, Dec 30, 2023 at 10:48 AM Apoorv Mittal 
> wrote:
>
> > Hi Stan,
> > Thanks for looking into the release. I worked with `./gradlewAll
> > publishToMavenLocal` which generates the respective `kafka-clients.jar`
> > and deploys to maven local, I believed that `./gradlewAll publish` should
> > just publish the artifacts to remote repository and hence should always
> > work as jars successfully gets deployed to local maven.
> >
> > Though now I set up the remote private maven repository for myself (on
> > jfrog) and tried `./gradlewAll publish` on the 3.7 branch and
> > successfully completed the build with all artifacts uploaded to the
> remote
> > repository. What seems strange to me is the error you mentioned in the
> > previous email regarding the reference of the clients jar. I suppose the
> > reference should be to `kafka-clients.jar` rather than `clients.jar`, I
> > might be missing if something else gets triggered in the release
> pipeline.
> > Do you think I should set up the remote repository as per the
> instructions
> > in `release.py` and try running `./release.py` as that might do something
> > different, though I suspect that it should?
> >
> > [image: Screenshot 2023-12-30 at 9.33.42 AM.png]
> >
> >
> > Regards,
> > Apoorv Mittal
> >
> >
> > On Fri, Dec 29, 2023 at 2:13 AM Colin McCabe  wrote:
> >
> >> Just to update this thread, everything in KAFKA-14127 is done now. A few
> >> tasks got moved to a separate umbrella JIRA.
> >>
> >> Some folks are going to do more testing, both manual and automated, in
> >> the next week or two. I think this will give us a good indicator of
> >> stability and what we need to fix.
> >>
> >> Right now I'm leaning towards just making it GA since that's how most
> >> features work. It's kind of rare for us to do a multi-step rollout for
> new
> >> features.
> >>
> >> best,
> >> Colin
> >>
> >>
> >> On Wed, Dec 20, 2023, at 03:43, Mickael Maison wrote:
> >> > Hi,
> >> >
> >> > With the current timeline for 3.7, I tend to agree with Viktor that
> >> > JBOD support in KRaft is unlikely to receive the extensive testing
> >> > this feature needs before releasing. And that's not counting the
> >> > testing tasks left to do in
> >> > https://issues.apache.org/jira/browse/KAFKA-14127.
> >> >
> >> > I'm fine sticking to the current 3.7 timeline but I'd err on the safe
> >> > side and mark JBOD as early access to avoid major issues. Kafka is
> >> > known for its robustness and resiliency and we certainly don't want to
> >> > lose the trust we gained over years.
> >> >
> >> > Thanks,
> >> > Mickael
> >> >
> >> > On Wed, Dec 20, 2023 at 12:24 AM Ismael Juma 
> wrote:
> >> >>
> >> >> Hi Viktor,
> >> >>
> >> >> Extending the code freeze doesn't help stabilize things. If we have
> >> >> important bugs for JBOD, we should mark those as blockers and we'll
> >> wait
> >> >> until they are fixed if the fixes won't take too long (as usual).
> >> >>
> >> >> Ismael
> >> >>
> >> >> On Tue, Dec 19, 2023 at 11:58 AM Viktor Somogyi-Vass
> >> >>  wrote:
> >> >>
> >> >> > Hi all,
> >> >> >
> >> >> > I was wondering what people think about extending the code freeze
> >> date to
> >> >> > early January?
> >> >> > The reason I'm asking is that there are still a couple of testing
> >> gaps in
> >> >> > JBOD (https://issues.apache.org/jira/browse/KAFKA-14127) which I
> >> think is
> >> >> > very important to finish to ensure a high quality release (after
> all
> >> this
> >> >> > supposed to be the last 3.x) and secondly the year end holidays for
> >> many
> >> >> > people are coming fast, which means we'll likely have less people
> >> working
> >> >> > on testing and validation. In my opinion it would strengthen the
> >> release if
> >> >> > we could spend a week in January to really finish off JBOD and do a
> >> 2 week
> >> >> > stabilization.
> >> >> >
> >> >> > What do you all think?
> >> >> >
> >> >> > Best,
> >> >> > Viktor
> >> >> >
> >> >> > On Tue, Dec 12, 2023 at 2:59 PM Stanislav Kozlovski
> >> >> >  wrote:
> >> >> >
> >> >> > > Hey!
> >> >> > >
> >> >> > > Just notifying everybody on this thread that I have cut the 3.7
> >> branch
> >> >> > and
> >> >> > > sent a new email thread titled "New Release Branch 3.7" to the
> >> mailing
> >> >> > list
> >> >> > > <
> 

[jira] [Reopened] (KAFKA-16046) Stream Stream Joins fail after restoration with deserialization exceptions

2024-01-02 Thread Almog Gavra (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-16046?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Almog Gavra reopened KAFKA-16046:
-

> Stream Stream Joins fail after restoration with deserialization exceptions
> --
>
> Key: KAFKA-16046
> URL: https://issues.apache.org/jira/browse/KAFKA-16046
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 3.7.0
>Reporter: Almog Gavra
>Assignee: Almog Gavra
>Priority: Blocker
>  Labels: streams
> Fix For: 3.7.0
>
>
> Before KIP-954, the `KStreamImplJoin` class would always create 
> non-timestamped persistent windowed stores. After that KIP, the default was 
> changed to create timestamped stores. This wasn't compatible because, during 
> restoration, timestamped stores have their changelog values transformed to 
> prepend the timestamp to the value. This caused serialization errors when 
> trying to read from the store because the deserializers did not expect the 
> timestamp to be prepended.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Jenkins build is still unstable: Kafka » Kafka Branch Builder » 3.7 #43

2024-01-02 Thread Apache Jenkins Server
See 




Re: [VOTE] KIP-1004: Enforce tasks.max property in Kafka Connect

2024-01-02 Thread Chris Egerton
Hi all,

Happy New Year! Wanted to give this a bump now that the holidays are over
for a lot of us. Looking forward to people's thoughts!

Cheers,

Chris

On Mon, Dec 4, 2023 at 10:36 AM Chris Egerton  wrote:

> Hi all,
>
> I'd like to call for a vote on KIP-1004, which adds enforcement for the
> tasks.max connector property in Kafka Connect.
>
> The KIP:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1004%3A+Enforce+tasks.max+property+in+Kafka+Connect
>
> The discussion thread:
> https://lists.apache.org/thread/scx75cjwm19jyt19wxky41q9smf5nx6d
>
> Cheers,
>
> Chris
>


Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #2528

2024-01-02 Thread Apache Jenkins Server
See 




Re: Kafka trunk test & build stability

2024-01-02 Thread Nick Telford
Addendum: I've opened a PR with what I believe are the changes necessary to
enable Remote Build Caching, if you choose to go that route:
https://github.com/apache/kafka/pull/15109

On Tue, 2 Jan 2024 at 14:31, Nick Telford  wrote:

> Hi everyone,
>
> Regarding building a "dependency graph"... Gradle already has this
> information, albeit fairly coarse-grained. You might be able to get some
> considerable improvement by configuring the Gradle Remote Build Cache. It
> looks like it's currently disabled explicitly:
> https://github.com/apache/kafka/blob/trunk/settings.gradle#L46
>
> The trick is to have trunk builds write to the cache, and PR builds only
> read from it. This way, any PR based on trunk should be able to cache not
> only the compilation, but also the tests from dependent modules that
> haven't changed (e.g. for a PR that only touches the connect/streams
> modules).
>
> This would probably be preferable to having to hand-maintain some
> rules/dependency graph in the CI configuration, and it's quite
> straight-forward to configure.
>
> Bonus points if the Remote Build Cache is readable publicly, enabling
> contributors to benefit from it locally.
>
> Regards,
> Nick
>
> On Tue, 2 Jan 2024 at 13:00, Lucas Brutschy 
> wrote:
>
>> Thanks for all the work that has already been done on this in the past
>> days!
>>
>> Have we considered running our test suite with
>> -XX:+HeapDumpOnOutOfMemoryError and uploading the heap dumps as
>> Jenkins build artifacts? This could speed up debugging. Even if we
>> store them only for a day and do it only for trunk, I think it could
>> be worth it. The heap dumps shouldn't contain any secrets, and I
>> checked with the ASF infra team, and they are not concerned about the
>> additional disk usage.
>>
>> Cheers,
>> Lucas
>>
>> On Wed, Dec 27, 2023 at 2:25 PM Divij Vaidya 
>> wrote:
>> >
>> > 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 was
>> curious if
>> > > folks had any ideas on how to identify the cause and fix the issue. I
>> was
>> > > looking in gradle enterprise and found some info about memory usage,
>> but
>> > > nothing detailed enough to help figure the issue out.
>> > >
>> > > OOMs sometimes fail the build immediately and in other cases I see it
>> get
>> > > stuck for 8 hours. (See
>> > >
>> > >
>> https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka/detail/trunk/2508/pipeline/12
>> > > )
>> > >
>> > > I appreciate all the work folks are doing here and I will continue to
>> try
>> > > to help as best as I can.
>> > >
>> > > Justine
>> > >
>> > > On Tue, Dec 26, 2023 at 1:04 PM David Arthur
>> > >  wrote:
>> > >
>> > > > S2. We’ve looked into this before, and it wasn’t possible at the
>> time
>> > > with
>> > > > JUnit. We commonly set a timeout on each test class (especially
>> > > integration
>> > > > tests). It is probably worth looking at this again and seeing if
>> > > something
>> > > > has changed with JUnit (or our usage of it) that would allow a
>> global
>> > > > timeout.
>> > > >
>> > > >
>> > > > S3. Dedicated infra sounds nice, if we can get it. It would at least
>> > > remove
>> > > > some variability between the builds, and hopefully eliminate the
>> > > > infra/setup class of failures.
>> > > >
>> > > >
>> > > > S4. Running tests for what has changed sounds nice, but I think it
>> is
>> > > risky
>> > > > to implement broadly. As Sophie mentioned, there are probably some
>> lines
>> > > we
>> > > > could draw where we feel confident that only running a subset of
>> tests is
>> > > > safe. As a start, we could probably work towards skipping CI for
>> non-code
>> > > > PRs.
>> > > >
>> > > >
>> > > > ---
>> > > >
>> > > >
>> > > > As an aside, I experimented with build caching and running affected
>> > > tests a
>> > > > few months ago. I used the opportunity to play with Github Actions,
>> and I
>> > > > quite liked it. Here’s the workflow I used:
>> > > >
>> https://github.com/mumrah/kafka/blob/trunk/.github/workflows/push.yml. I
>> > > > was trying to see if we could use a build cache to reduce the
>> compilation
>> > > > time on PRs. A nightly/periodic job would build trunk and populate a
>> > > Gradle
>> > > > build cache. PR builds would read from that cache which would
>> enable them
>> > > > to only compile changed code. The same idea could be extended to
>> tests,
>> > > but
>> > > > I didn’t get that far.
>> > > >
>> > > >
>> > > > As for Github Actions, the idea there is that ASF would provide
>> generic
>> > > > Action “runners” that would pick up jobs from the Github Action
>> build
>> > > queue
>> > > > and run them. It is also possible to self-host runners to expand the
>> > > build
>> > > > capacity of 

Re: Kafka trunk test & build stability

2024-01-02 Thread Nick Telford
Hi everyone,

Regarding building a "dependency graph"... Gradle already has this
information, albeit fairly coarse-grained. You might be able to get some
considerable improvement by configuring the Gradle Remote Build Cache. It
looks like it's currently disabled explicitly:
https://github.com/apache/kafka/blob/trunk/settings.gradle#L46

The trick is to have trunk builds write to the cache, and PR builds only
read from it. This way, any PR based on trunk should be able to cache not
only the compilation, but also the tests from dependent modules that
haven't changed (e.g. for a PR that only touches the connect/streams
modules).

This would probably be preferable to having to hand-maintain some
rules/dependency graph in the CI configuration, and it's quite
straight-forward to configure.

Bonus points if the Remote Build Cache is readable publicly, enabling
contributors to benefit from it locally.

Regards,
Nick

On Tue, 2 Jan 2024 at 13:00, Lucas Brutschy 
wrote:

> Thanks for all the work that has already been done on this in the past
> days!
>
> Have we considered running our test suite with
> -XX:+HeapDumpOnOutOfMemoryError and uploading the heap dumps as
> Jenkins build artifacts? This could speed up debugging. Even if we
> store them only for a day and do it only for trunk, I think it could
> be worth it. The heap dumps shouldn't contain any secrets, and I
> checked with the ASF infra team, and they are not concerned about the
> additional disk usage.
>
> Cheers,
> Lucas
>
> On Wed, Dec 27, 2023 at 2:25 PM Divij Vaidya 
> wrote:
> >
> > 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 was
> curious if
> > > folks had any ideas on how to identify the cause and fix the issue. I
> was
> > > looking in gradle enterprise and found some info about memory usage,
> but
> > > nothing detailed enough to help figure the issue out.
> > >
> > > OOMs sometimes fail the build immediately and in other cases I see it
> get
> > > stuck for 8 hours. (See
> > >
> > >
> https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka/detail/trunk/2508/pipeline/12
> > > )
> > >
> > > I appreciate all the work folks are doing here and I will continue to
> try
> > > to help as best as I can.
> > >
> > > Justine
> > >
> > > On Tue, Dec 26, 2023 at 1:04 PM David Arthur
> > >  wrote:
> > >
> > > > S2. We’ve looked into this before, and it wasn’t possible at the time
> > > with
> > > > JUnit. We commonly set a timeout on each test class (especially
> > > integration
> > > > tests). It is probably worth looking at this again and seeing if
> > > something
> > > > has changed with JUnit (or our usage of it) that would allow a global
> > > > timeout.
> > > >
> > > >
> > > > S3. Dedicated infra sounds nice, if we can get it. It would at least
> > > remove
> > > > some variability between the builds, and hopefully eliminate the
> > > > infra/setup class of failures.
> > > >
> > > >
> > > > S4. Running tests for what has changed sounds nice, but I think it is
> > > risky
> > > > to implement broadly. As Sophie mentioned, there are probably some
> lines
> > > we
> > > > could draw where we feel confident that only running a subset of
> tests is
> > > > safe. As a start, we could probably work towards skipping CI for
> non-code
> > > > PRs.
> > > >
> > > >
> > > > ---
> > > >
> > > >
> > > > As an aside, I experimented with build caching and running affected
> > > tests a
> > > > few months ago. I used the opportunity to play with Github Actions,
> and I
> > > > quite liked it. Here’s the workflow I used:
> > > >
> https://github.com/mumrah/kafka/blob/trunk/.github/workflows/push.yml. I
> > > > was trying to see if we could use a build cache to reduce the
> compilation
> > > > time on PRs. A nightly/periodic job would build trunk and populate a
> > > Gradle
> > > > build cache. PR builds would read from that cache which would enable
> them
> > > > to only compile changed code. The same idea could be extended to
> tests,
> > > but
> > > > I didn’t get that far.
> > > >
> > > >
> > > > As for Github Actions, the idea there is that ASF would provide
> generic
> > > > Action “runners” that would pick up jobs from the Github Action build
> > > queue
> > > > and run them. It is also possible to self-host runners to expand the
> > > build
> > > > capacity of the project (i.e., other organizations could donate
> > > > build capacity). The advantage of this is that we would have more
> control
> > > > over our build/reports and not be “stuck” with whatever ASF Jenkins
> > > offers.
> > > > The Actions workflows are very customizable and it would let us
> create
> > > our
> > > > own custom plugins. There is also a substantial marketplace of
> plugins. I
> > 

Re: Kafka trunk test & build stability

2024-01-02 Thread Lucas Brutschy
Thanks for all the work that has already been done on this in the past days!

Have we considered running our test suite with
-XX:+HeapDumpOnOutOfMemoryError and uploading the heap dumps as
Jenkins build artifacts? This could speed up debugging. Even if we
store them only for a day and do it only for trunk, I think it could
be worth it. The heap dumps shouldn't contain any secrets, and I
checked with the ASF infra team, and they are not concerned about the
additional disk usage.

Cheers,
Lucas

On Wed, Dec 27, 2023 at 2:25 PM Divij Vaidya  wrote:
>
> 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 was curious if
> > folks had any ideas on how to identify the cause and fix the issue. I was
> > looking in gradle enterprise and found some info about memory usage, but
> > nothing detailed enough to help figure the issue out.
> >
> > OOMs sometimes fail the build immediately and in other cases I see it get
> > stuck for 8 hours. (See
> >
> > https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka/detail/trunk/2508/pipeline/12
> > )
> >
> > I appreciate all the work folks are doing here and I will continue to try
> > to help as best as I can.
> >
> > Justine
> >
> > On Tue, Dec 26, 2023 at 1:04 PM David Arthur
> >  wrote:
> >
> > > S2. We’ve looked into this before, and it wasn’t possible at the time
> > with
> > > JUnit. We commonly set a timeout on each test class (especially
> > integration
> > > tests). It is probably worth looking at this again and seeing if
> > something
> > > has changed with JUnit (or our usage of it) that would allow a global
> > > timeout.
> > >
> > >
> > > S3. Dedicated infra sounds nice, if we can get it. It would at least
> > remove
> > > some variability between the builds, and hopefully eliminate the
> > > infra/setup class of failures.
> > >
> > >
> > > S4. Running tests for what has changed sounds nice, but I think it is
> > risky
> > > to implement broadly. As Sophie mentioned, there are probably some lines
> > we
> > > could draw where we feel confident that only running a subset of tests is
> > > safe. As a start, we could probably work towards skipping CI for non-code
> > > PRs.
> > >
> > >
> > > ---
> > >
> > >
> > > As an aside, I experimented with build caching and running affected
> > tests a
> > > few months ago. I used the opportunity to play with Github Actions, and I
> > > quite liked it. Here’s the workflow I used:
> > > https://github.com/mumrah/kafka/blob/trunk/.github/workflows/push.yml. I
> > > was trying to see if we could use a build cache to reduce the compilation
> > > time on PRs. A nightly/periodic job would build trunk and populate a
> > Gradle
> > > build cache. PR builds would read from that cache which would enable them
> > > to only compile changed code. The same idea could be extended to tests,
> > but
> > > I didn’t get that far.
> > >
> > >
> > > As for Github Actions, the idea there is that ASF would provide generic
> > > Action “runners” that would pick up jobs from the Github Action build
> > queue
> > > and run them. It is also possible to self-host runners to expand the
> > build
> > > capacity of the project (i.e., other organizations could donate
> > > build capacity). The advantage of this is that we would have more control
> > > over our build/reports and not be “stuck” with whatever ASF Jenkins
> > offers.
> > > The Actions workflows are very customizable and it would let us create
> > our
> > > own custom plugins. There is also a substantial marketplace of plugins. I
> > > think it’s worth exploring this more, I just haven’t had time lately.
> > >
> > > On Tue, Dec 26, 2023 at 3:24 PM Sophie Blee-Goldman <
> > sop...@responsive.dev
> > > >
> > > wrote:
> > >
> > > > Regarding:
> > > >
> > > > S-4. Separate tests ran depending on what module is changed.
> > > > >
> > > > - This makes sense although is tricky to implement successfully, as
> > > > > unrelated tests may expose problems in an unrelated change (e.g
> > > changing
> > > > > core stuff like clients, the server, etc)
> > > >
> > > >
> > > > Imo this avenue could provide a massive improvement to dev productivity
> > > > with very little effort or investment, and if we do it right, without
> > > even
> > > > any risk. We should be able to draft a simple dependency graph between
> > > > modules and then skip the tests for anything that is clearly, provably
> > > > unrelated and/or upstream of the target changes. This has the potential
> > > to
> > > > substantially speed up and improve the developer experience in modules
> > at
> > > > the end of the dependency graph, which I believe is worth doing even if
> > > it
> > > > unfortunately would not benefit everyone equally.
> > > >
> > > > For example, we can 

Re: Contributor permissions

2024-01-02 Thread Aadithya C
Thanks Justine!

On Mon, Jan 1, 2024 at 8:24 AM Justine Olshan 
wrote:

> It is also possible the name you chose was already taken. I have emailed
> infra to clarify.
>
> On Mon, Jan 1, 2024 at 8:19 AM Justine Olshan 
> wrote:
>
> > Hey Aadithya,
> >
> > I saw your request and tried to approve it but there was a conflict with
> > the name of the account. It was asking me to contact Infra.
> >
> > I can try to do that.
> >
> > Thanks,
> > Justine
> >
> > On Sun, Dec 31, 2023 at 2:12 PM Aadithya C  wrote:
> >
> >> I created a new username - *aadithya*. Please let me know if this one
> >> works
> >> for you.
> >>
> >> Adithya
> >>
> >>
> >> On Sun, Dec 31, 2023 at 2:00 PM Aadithya C 
> wrote:
> >>
> >> > Thanks! I submitted a request to retrieve my username but I havent
> >> > received any email from the app. Is there a different way to retrieve
> >> > my user-id?
> >> >
> >> >
> >> > On Fri, Dec 29, 2023 at 9:11 AM Greg Harris
> >> >  wrote:
> >> > >
> >> > > Hey Adithya,
> >> > >
> >> > > Thanks for your interest in Apache Kafka!
> >> > > I added permissions for your Wiki account, so you should be ready to
> >> > > contribute KIPs.
> >> > >
> >> > > I could not find your ASF JIRA account though. Could you verify that
> >> > > is the username shown in the Jira dashboard?
> >> > > https://issues.apache.org/jira/secure/ViewProfile.jspa
> >> > >
> >> > > Thanks!
> >> > > Greg
> >> > >
> >> > > On Thu, Dec 28, 2023 at 1:21 PM Aadithya C 
> >> wrote:
> >> > > >
> >> > > > Requesting contributor permissions.
> >> > > >
> >> > > > JIRA username: ditac
> >> > > > GitHub username: ditac
> >> > > > Wiki username: ditac
> >> > > >
> >> > > >
> >> > > > Thanks,
> >> > > > Adithya
> >> >
> >>
> >
>


Re: Apache Kafka 3.7.0 Release

2024-01-02 Thread Stanislav Kozlovski
Hi Apoorv,

Thanks for taking ownership and looking into this! One more caveat is that
I believe this first publish is ran with JDK 8, as the release.py runs with
both JDK 8 and (if I recall correctly) 17 versions. This seems to fail on
the first one - so JDK 8.
Not sure if that is related in any way. And I'm also not sure if it should
be kafka-clients or just clients.

On Sat, Dec 30, 2023 at 10:48 AM Apoorv Mittal 
wrote:

> Hi Stan,
> Thanks for looking into the release. I worked with `./gradlewAll
> publishToMavenLocal` which generates the respective `kafka-clients.jar`
> and deploys to maven local, I believed that `./gradlewAll publish` should
> just publish the artifacts to remote repository and hence should always
> work as jars successfully gets deployed to local maven.
>
> Though now I set up the remote private maven repository for myself (on
> jfrog) and tried `./gradlewAll publish` on the 3.7 branch and
> successfully completed the build with all artifacts uploaded to the remote
> repository. What seems strange to me is the error you mentioned in the
> previous email regarding the reference of the clients jar. I suppose the
> reference should be to `kafka-clients.jar` rather than `clients.jar`, I
> might be missing if something else gets triggered in the release pipeline.
> Do you think I should set up the remote repository as per the instructions
> in `release.py` and try running `./release.py` as that might do something
> different, though I suspect that it should?
>
> [image: Screenshot 2023-12-30 at 9.33.42 AM.png]
>
>
> Regards,
> Apoorv Mittal
>
>
> On Fri, Dec 29, 2023 at 2:13 AM Colin McCabe  wrote:
>
>> Just to update this thread, everything in KAFKA-14127 is done now. A few
>> tasks got moved to a separate umbrella JIRA.
>>
>> Some folks are going to do more testing, both manual and automated, in
>> the next week or two. I think this will give us a good indicator of
>> stability and what we need to fix.
>>
>> Right now I'm leaning towards just making it GA since that's how most
>> features work. It's kind of rare for us to do a multi-step rollout for new
>> features.
>>
>> best,
>> Colin
>>
>>
>> On Wed, Dec 20, 2023, at 03:43, Mickael Maison wrote:
>> > Hi,
>> >
>> > With the current timeline for 3.7, I tend to agree with Viktor that
>> > JBOD support in KRaft is unlikely to receive the extensive testing
>> > this feature needs before releasing. And that's not counting the
>> > testing tasks left to do in
>> > https://issues.apache.org/jira/browse/KAFKA-14127.
>> >
>> > I'm fine sticking to the current 3.7 timeline but I'd err on the safe
>> > side and mark JBOD as early access to avoid major issues. Kafka is
>> > known for its robustness and resiliency and we certainly don't want to
>> > lose the trust we gained over years.
>> >
>> > Thanks,
>> > Mickael
>> >
>> > On Wed, Dec 20, 2023 at 12:24 AM Ismael Juma  wrote:
>> >>
>> >> Hi Viktor,
>> >>
>> >> Extending the code freeze doesn't help stabilize things. If we have
>> >> important bugs for JBOD, we should mark those as blockers and we'll
>> wait
>> >> until they are fixed if the fixes won't take too long (as usual).
>> >>
>> >> Ismael
>> >>
>> >> On Tue, Dec 19, 2023 at 11:58 AM Viktor Somogyi-Vass
>> >>  wrote:
>> >>
>> >> > Hi all,
>> >> >
>> >> > I was wondering what people think about extending the code freeze
>> date to
>> >> > early January?
>> >> > The reason I'm asking is that there are still a couple of testing
>> gaps in
>> >> > JBOD (https://issues.apache.org/jira/browse/KAFKA-14127) which I
>> think is
>> >> > very important to finish to ensure a high quality release (after all
>> this
>> >> > supposed to be the last 3.x) and secondly the year end holidays for
>> many
>> >> > people are coming fast, which means we'll likely have less people
>> working
>> >> > on testing and validation. In my opinion it would strengthen the
>> release if
>> >> > we could spend a week in January to really finish off JBOD and do a
>> 2 week
>> >> > stabilization.
>> >> >
>> >> > What do you all think?
>> >> >
>> >> > Best,
>> >> > Viktor
>> >> >
>> >> > On Tue, Dec 12, 2023 at 2:59 PM Stanislav Kozlovski
>> >> >  wrote:
>> >> >
>> >> > > Hey!
>> >> > >
>> >> > > Just notifying everybody on this thread that I have cut the 3.7
>> branch
>> >> > and
>> >> > > sent a new email thread titled "New Release Branch 3.7" to the
>> mailing
>> >> > list
>> >> > > > >.
>> >> > >
>> >> > > Best,
>> >> > > Stanislav
>> >> > >
>> >> > > On Wed, Dec 6, 2023 at 11:10 AM Stanislav Kozlovski <
>> >> > > stanis...@confluent.io>
>> >> > > wrote:
>> >> > >
>> >> > > > Hello again,
>> >> > > >
>> >> > > > Time is flying by! It is feature freeze day!
>> >> > > >
>> >> > > > By today, we expect to have major features merged and to begin
>> working
>> >> > on
>> >> > > > their stabilisation. Minor features should have PRs.
>> >> > > >
>> >> > > > I am planning to cut the release branch soon - on 

Re: [DISCUSS] Kafka distributed transaction support

2024-01-02 Thread Andrew Schofield
Hi Rowland,
KIP-939 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-939%3A+Support+Participation+in+2PC
introduces two-phase commit, but not XA support. I believe the KIP is expected 
to be delivered in
Kafka 4.0 later this year. The way that it works and the use cases that it’s 
intended for were carefully
considered and it intentionally does not support XA.

There would be a considerable amount of work for full XA support, in particular 
when you start to include
other related technologies such as Java EE resource adapters that most people 
use in practice to exploit XA.
I wonder whether the KIP-939 changes would make it viable to implement 
XAResource on top of the Kafka
client for some specific scenarios at a much lower cost than full support.

Personally, as a non-PMC member, I would not advocate trying to put full XA 
support into Kafka.

Thanks,
Andrew

> On 1 Jan 2024, at 22:20, Rowland Smith  wrote:
> 
> Hi All,
> 
> I am new to the Kafka developer community. After learning more about
> Kafka's transactional capabilities recently, I became interested in
> exploring what would be required to provide support for the XA interface
> specified in the X/ Open Distributed Processing Model in the Kafka producer
> client. I have put together a proof of concept to satisfy my curiosity, and
> based on that work, I think that extending the Kafka producer with XA
> support is doable with reasonable effort.
> 
> As I understand the Kafka development team's process, the first step in the
> process would be to produce a KIP describing the feature's goals and
> design. My question in this email is whether XA support has ever been
> considered previously by the PMC and if so, with what result. I don't want
> to spend time working on a KIP if XA support is not something that the PMC
> sees value in including and supporting in the Kafka codebase.
> 
> Any feedback would be appreciated. I am excited to work on this feature if
> there is interest in the community.
> 
> Regards,
> Rowland



[jira] [Created] (KAFKA-16074) Fix thread leaks in ReplicaManagerTest

2024-01-02 Thread Divij Vaidya (Jira)
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
Reporter: Luke Chen
Assignee: Luke Chen


Following [@dajac|https://github.com/dajac] 's finding in 
[#15063|https://github.com/apache/kafka/pull/15063], I found we also create new 
RemoteLogManager in ReplicaManagerTest, but didn't close them.

While investigating ReplicaManagerTest, I also found there are other threads 
leaking:
 # remote fetch reaper thread. It's because we create a reaper thread in test, 
which is not expected. We should create a mocked one like other purgatory 
instance.
 # Throttle threads. We created a {{quotaManager}} to feed into the 
replicaManager, but didn't close it. Actually, we have created a global 
{{quotaManager}} instance and will close it on {{{}AfterEach{}}}. We should 
re-use it.
 # replicaManager and logManager didn't invoke {{close}} after test.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


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

2024-01-02 Thread Divij Vaidya
Thanks for the KIP Christo.

The shell command that you mentioned calls ListOffsets API internally.
Hence, I believe that we would be making a public interface change (and a
version bump) to ListOffsetsAPI as well to include -5? If yes, can you
please add that information to the change in public 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 explanations regarding the changes needed to the OffsetSpec or
> others.
>
> Best,
> Christo
>
> On Tue, 21 Nov 2023 at 04:20, Satish Duggana 
> wrote:
>
> > Thanks Christo for starting the discussion on the KIP.
> >
> > As mentioned in KAFKA-15857[1], the goal is to add new entries for
> > local-log-start-offset and tierd-offset in OffsetSpec. This will be
> > used in AdminClient APIs and also to be added as part of
> > GetOffsetShell. This was also raised by Kamal in the earlier email.
> >
> > OffsetSpec related changes for these entries also need to be mentioned
> > as part of the PublicInterfaces section because these are exposed to
> > users as public APIs through Admin#listOffsets() APIs[2, 3].
> >
> > Please update the KIP with the above details.
> >
> > 1. https://issues.apache.org/jira/browse/KAFKA-15857
> > 2.
> >
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java#L1238
> > 3.
> >
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java#L1226
> >
> > ~Satish.
> >
> > On Mon, 20 Nov 2023 at 18:35, Kamal Chandraprakash
> >  wrote:
> > >
> > > 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 a discussion for
> > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1005%3A+Add+EarliestLocalOffset+to+GetOffsetShell
> > > > .
> > > >
> > > > A new offset called local log start offset was introduced as part of
> > > > KIP-405: Kafka Tiered Storage. KIP-1005 aims to expose this offset by
> > > > changing the AdminClient and in particular the GetOffsetShell tool.
> > > >
> > > > I am looking forward to your suggestions for improvement!
> > > >
> > > > Best,
> > > > Christo
> > > >
> >
>


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

2024-01-02 Thread Matthias J. Sax

Congrats!

On 12/29/23 9:40 AM, Viktor Somogyi-Vass wrote:

Congrats Divij, well deserved!

On Fri, Dec 29, 2023, 09:36 Lucas Brutschy 
wrote:


Congratulations, Divij!

On Fri, Dec 29, 2023 at 1:32 AM Colin McCabe  wrote:


Congratulations, Divij!

best,
Colin

On Thu, Dec 28, 2023, at 11:38, Bruno Cadonna wrote:

Congratulations Divij! Well deserved!

Best,
Bruno

On 12/27/23 12:45 PM, Luke Chen wrote:

Hi, Everyone,

Divij has been a Kafka committer since June, 2023. He has remained

very

active and instructive in the community since becoming a committer.

It's my

pleasure to announce that Divij is now a member of Kafka PMC.

Congratulations Divij!

Luke
on behalf of Apache Kafka PMC







Re: [DISCUSS] Connect Jira component name

2024-01-02 Thread Matthias J. Sax

SGTM.

I found it always odd that it's named `KafkaConnect` :)


-Matthias

On 12/26/23 6:03 PM, Greg Harris wrote:

Hi Connect Developers,

I noticed recently that we had two Jira components: "KafkaConnect" and
"connect", one with >1000 issues and one with 20 issues. I merged the
two tags, leaving the one labeled "KafkaConnect".

"KafkaConnect" doesn't follow the standard naming convention set by
all of the other components, while "connect" does. Should we rename
"KafkaConnect" to "connect" moving forward?

Forgive me for bikeshedding,
Greg