Re: [DISCUSS] KIP-852 Optimize calculation of size for log in remote tier

2023-06-30 Thread Kamal Chandraprakash
Hi Divij,

Thanks for the explanation. LGTM.

--
Kamal

On Sat, Jul 1, 2023 at 7:28 AM Satish Duggana 
wrote:

> Hi Divij,
> I am fine with having an API to compute the size as I mentioned in my
> earlier reply in this mail thread. But I have the below comment for
> the motivation for this KIP.
>
> As you discussed offline, the main issue here is listing calls for
> remote log segment metadata is slower because of the storage used for
> RLMM. These can be avoided with this new API.
>
> Please add this in the motivation section as it is one of the main
> motivations for the KIP.
>
> Thanks,
> Satish.
>
> On Sat, 1 Jul 2023 at 01:43, Jun Rao  wrote:
> >
> > Hi, Divij,
> >
> > Sorry for the late reply.
> >
> > Given your explanation, the new API sounds reasonable to me. Is that
> enough
> > to build the external metadata layer for the remote segments or do you
> need
> > some additional API changes?
> >
> > Thanks,
> >
> > Jun
> >
> > On Fri, Jun 9, 2023 at 7:08 AM Divij Vaidya 
> wrote:
> >
> > > Thank you for looking into this Kamal.
> > >
> > > You are right in saying that a cold start (i.e. leadership failover or
> > > broker startup) does not impact the broker startup duration. But it
> does
> > > have the following impact:
> > > 1. It leads to a burst of full-scan requests to RLMM in case multiple
> > > leadership failovers occur at the same time. Even if the RLMM
> > > implementation has the capability to serve the total size from an index
> > > (and hence handle this burst), we wouldn't be able to use it since the
> > > current API necessarily calls for a full scan.
> > > 2. The archival (copying of data to tiered storage) process will have a
> > > delayed start. The delayed start of archival could lead to local build
> up
> > > of data which may lead to disk full.
> > >
> > > The disadvantage of adding this new API is that every provider will
> have to
> > > implement it, agreed. But I believe that this tradeoff is worthwhile
> since
> > > the default implementation could be the same as you mentioned, i.e.
> keeping
> > > cumulative in-memory count.
> > >
> > > --
> > > Divij Vaidya
> > >
> > >
> > >
> > > On Sun, Jun 4, 2023 at 5:48 PM Kamal Chandraprakash <
> > > kamal.chandraprak...@gmail.com> wrote:
> > >
> > > > Hi Divij,
> > > >
> > > > Thanks for the KIP! Sorry for the late reply.
> > > >
> > > > Can you explain the rejected alternative-3?
> > > > Store the cumulative size of remote tier log in-memory at
> > > RemoteLogManager
> > > > "*Cons*: Every time a broker starts-up, it will scan through all the
> > > > segments in the remote tier to initialise the in-memory value. This
> would
> > > > increase the broker start-up time."
> > > >
> > > > Keeping the source of truth to determine the remote-log-size in the
> > > leader
> > > > would be consistent across different implementations of the plugin.
> The
> > > > concern posted in the KIP is that we are calculating the
> remote-log-size
> > > on
> > > > each iteration of the cleaner thread (say 5 mins). If we calculate
> only
> > > > once during broker startup or during the leadership reassignment, do
> we
> > > > still need the cache?
> > > >
> > > > The broker startup-time won't be affected by the remote log manager
> > > > initialisation. The broker continue to start accepting the new
> > > > produce/fetch requests, while the RLM thread in the background can
> > > > determine the remote-log-size once and start copying/deleting the
> > > segments.
> > > >
> > > > Thanks,
> > > > Kamal
> > > >
> > > > On Thu, Jun 1, 2023 at 2:08 PM Divij Vaidya  >
> > > > wrote:
> > > >
> > > > > Satish / Jun
> > > > >
> > > > > Do you have any thoughts on this?
> > > > >
> > > > > --
> > > > > Divij Vaidya
> > > > >
> > > > >
> > > > >
> > > > > On Tue, Feb 14, 2023 at 4:15 PM Divij Vaidya <
> divijvaidy...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hey Jun
> > > > > >
> > > > > > It has been a while since this KIP got some attention. While we
> wait
> > > > for
> > > > > > Satish to chime in here, perhaps I can answer your question.
> > > > > >
> > > > > > > Could you explain how you exposed the log size in your KIP-405
> > > > > > implementation?
> > > > > >
> > > > > > The APIs available in RLMM as per KIP405
> > > > > > are, addRemoteLogSegmentMetadata(),
> updateRemoteLogSegmentMetadata(),
> > > > > remoteLogSegmentMetadata(), highestOffsetForEpoch(),
> > > > > putRemotePartitionDeleteMetadata(), listRemoteLogSegments(),
> > > > > onPartitionLeadershipChanges()
> > > > > > and onStopPartitions(). None of these APIs allow us to expose
> the log
> > > > > size,
> > > > > > hence, the only option that remains is to list all segments using
> > > > > > listRemoteLogSegments() and aggregate them every time we require
> to
> > > > > > calculate the size. Based on our prior discussion, this requires
> > > > reading
> > > > > > all segment metadata which won't work for non-local RLMM
> > > > implementations.
> > > > > > Satish's implementation also 

Re: [DISCUSS] KIP-852 Optimize calculation of size for log in remote tier

2023-06-30 Thread Satish Duggana
Hi Divij,
I am fine with having an API to compute the size as I mentioned in my
earlier reply in this mail thread. But I have the below comment for
the motivation for this KIP.

As you discussed offline, the main issue here is listing calls for
remote log segment metadata is slower because of the storage used for
RLMM. These can be avoided with this new API.

Please add this in the motivation section as it is one of the main
motivations for the KIP.

Thanks,
Satish.

On Sat, 1 Jul 2023 at 01:43, Jun Rao  wrote:
>
> Hi, Divij,
>
> Sorry for the late reply.
>
> Given your explanation, the new API sounds reasonable to me. Is that enough
> to build the external metadata layer for the remote segments or do you need
> some additional API changes?
>
> Thanks,
>
> Jun
>
> On Fri, Jun 9, 2023 at 7:08 AM Divij Vaidya  wrote:
>
> > Thank you for looking into this Kamal.
> >
> > You are right in saying that a cold start (i.e. leadership failover or
> > broker startup) does not impact the broker startup duration. But it does
> > have the following impact:
> > 1. It leads to a burst of full-scan requests to RLMM in case multiple
> > leadership failovers occur at the same time. Even if the RLMM
> > implementation has the capability to serve the total size from an index
> > (and hence handle this burst), we wouldn't be able to use it since the
> > current API necessarily calls for a full scan.
> > 2. The archival (copying of data to tiered storage) process will have a
> > delayed start. The delayed start of archival could lead to local build up
> > of data which may lead to disk full.
> >
> > The disadvantage of adding this new API is that every provider will have to
> > implement it, agreed. But I believe that this tradeoff is worthwhile since
> > the default implementation could be the same as you mentioned, i.e. keeping
> > cumulative in-memory count.
> >
> > --
> > Divij Vaidya
> >
> >
> >
> > On Sun, Jun 4, 2023 at 5:48 PM Kamal Chandraprakash <
> > kamal.chandraprak...@gmail.com> wrote:
> >
> > > Hi Divij,
> > >
> > > Thanks for the KIP! Sorry for the late reply.
> > >
> > > Can you explain the rejected alternative-3?
> > > Store the cumulative size of remote tier log in-memory at
> > RemoteLogManager
> > > "*Cons*: Every time a broker starts-up, it will scan through all the
> > > segments in the remote tier to initialise the in-memory value. This would
> > > increase the broker start-up time."
> > >
> > > Keeping the source of truth to determine the remote-log-size in the
> > leader
> > > would be consistent across different implementations of the plugin. The
> > > concern posted in the KIP is that we are calculating the remote-log-size
> > on
> > > each iteration of the cleaner thread (say 5 mins). If we calculate only
> > > once during broker startup or during the leadership reassignment, do we
> > > still need the cache?
> > >
> > > The broker startup-time won't be affected by the remote log manager
> > > initialisation. The broker continue to start accepting the new
> > > produce/fetch requests, while the RLM thread in the background can
> > > determine the remote-log-size once and start copying/deleting the
> > segments.
> > >
> > > Thanks,
> > > Kamal
> > >
> > > On Thu, Jun 1, 2023 at 2:08 PM Divij Vaidya 
> > > wrote:
> > >
> > > > Satish / Jun
> > > >
> > > > Do you have any thoughts on this?
> > > >
> > > > --
> > > > Divij Vaidya
> > > >
> > > >
> > > >
> > > > On Tue, Feb 14, 2023 at 4:15 PM Divij Vaidya 
> > > > wrote:
> > > >
> > > > > Hey Jun
> > > > >
> > > > > It has been a while since this KIP got some attention. While we wait
> > > for
> > > > > Satish to chime in here, perhaps I can answer your question.
> > > > >
> > > > > > Could you explain how you exposed the log size in your KIP-405
> > > > > implementation?
> > > > >
> > > > > The APIs available in RLMM as per KIP405
> > > > > are, addRemoteLogSegmentMetadata(), updateRemoteLogSegmentMetadata(),
> > > > remoteLogSegmentMetadata(), highestOffsetForEpoch(),
> > > > putRemotePartitionDeleteMetadata(), listRemoteLogSegments(),
> > > > onPartitionLeadershipChanges()
> > > > > and onStopPartitions(). None of these APIs allow us to expose the log
> > > > size,
> > > > > hence, the only option that remains is to list all segments using
> > > > > listRemoteLogSegments() and aggregate them every time we require to
> > > > > calculate the size. Based on our prior discussion, this requires
> > > reading
> > > > > all segment metadata which won't work for non-local RLMM
> > > implementations.
> > > > > Satish's implementation also performs a full scan and calculates the
> > > > > aggregate. see:
> > > > >
> > > >
> > >
> > https://github.com/satishd/kafka/blob/2.8.x-tiered-storage/core/src/main/scala/kafka/log/remote/RemoteLogManager.scala#L619
> > > > >
> > > > >
> > > > > Does this answer your question?
> > > > >
> > > > > --
> > > > > Divij Vaidya
> > > > >
> > > > >
> > > > >
> > > > > On Tue, Dec 20, 2022 at 8:40 PM Jun Rao 
> > > > wrote:
> > > 

[jira] [Created] (KAFKA-15138) Java kafka-clients compression dependencies should be optional

2023-06-30 Thread Joe DiPol (Jira)
Joe DiPol created KAFKA-15138:
-

 Summary: Java kafka-clients compression dependencies should be 
optional
 Key: KAFKA-15138
 URL: https://issues.apache.org/jira/browse/KAFKA-15138
 Project: Kafka
  Issue Type: Bug
  Components: clients
Affects Versions: 3.4.0
Reporter: Joe DiPol


If you look at

[https://repo1.maven.org/maven2/org/apache/kafka/kafka-clients/3.4.0/kafka-clients-3.4.0.pom]

You see that the dependencies for the compression libraries (like lz4-java) do 
NOT have "{{{}true{}}}". That means that these libraries 
are transitive dependencies which will be pulled (and potentially security 
scanned) for any project that uses kafka-clients. 

This is not correct. These compression libraries are optional and should not be 
transitive dependencies of kafka-clients. Therefore the above pom should state 
{{optional}} like:


    org.lz4
    lz4-java
    1.8.0
    runtime
    true




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


Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #1965

2023-06-30 Thread Apache Jenkins Server
See 


Changes:


--
[...truncated 385295 lines...]
Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testLeftWithVersionedStores[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testOuterWithLeftVersionedOnly[caching enabled = false] STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testOuterWithLeftVersionedOnly[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testLeftWithRightVersionedOnly[caching enabled = false] STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testLeftWithRightVersionedOnly[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerInner[caching enabled = false] STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerInner[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerOuter[caching enabled = false] STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerOuter[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerLeft[caching enabled = false] STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerLeft[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testOuterInner[caching enabled = false] STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testOuterInner[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testOuterOuter[caching enabled = false] STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testOuterOuter[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerWithRightVersionedOnly[caching enabled = false] STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerWithRightVersionedOnly[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testLeftWithLeftVersionedOnly[caching enabled = false] STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testLeftWithLeftVersionedOnly[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerWithLeftVersionedOnly[caching enabled = false] STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TableTableJoinIntegrationTest > [caching enabled = false] > 
testInnerWithLeftVersionedOnly[caching enabled = false] PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TaskAssignorIntegrationTest > shouldProperlyConfigureTheAssignor STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TaskAssignorIntegrationTest > shouldProperlyConfigureTheAssignor PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TaskMetadataIntegrationTest > shouldReportCorrectEndOffsetInformation STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TaskMetadataIntegrationTest > shouldReportCorrectEndOffsetInformation PASSED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TaskMetadataIntegrationTest > shouldReportCorrectCommittedOffsetInformation 
STARTED

Gradle Test Run :streams:integrationTest > Gradle Test Executor 183 > 
TaskMetadataIntegrationTest > 

Re: [DISCUSS] KIP-852 Optimize calculation of size for log in remote tier

2023-06-30 Thread Jun Rao
Hi, Divij,

Sorry for the late reply.

Given your explanation, the new API sounds reasonable to me. Is that enough
to build the external metadata layer for the remote segments or do you need
some additional API changes?

Thanks,

Jun

On Fri, Jun 9, 2023 at 7:08 AM Divij Vaidya  wrote:

> Thank you for looking into this Kamal.
>
> You are right in saying that a cold start (i.e. leadership failover or
> broker startup) does not impact the broker startup duration. But it does
> have the following impact:
> 1. It leads to a burst of full-scan requests to RLMM in case multiple
> leadership failovers occur at the same time. Even if the RLMM
> implementation has the capability to serve the total size from an index
> (and hence handle this burst), we wouldn't be able to use it since the
> current API necessarily calls for a full scan.
> 2. The archival (copying of data to tiered storage) process will have a
> delayed start. The delayed start of archival could lead to local build up
> of data which may lead to disk full.
>
> The disadvantage of adding this new API is that every provider will have to
> implement it, agreed. But I believe that this tradeoff is worthwhile since
> the default implementation could be the same as you mentioned, i.e. keeping
> cumulative in-memory count.
>
> --
> Divij Vaidya
>
>
>
> On Sun, Jun 4, 2023 at 5:48 PM Kamal Chandraprakash <
> kamal.chandraprak...@gmail.com> wrote:
>
> > Hi Divij,
> >
> > Thanks for the KIP! Sorry for the late reply.
> >
> > Can you explain the rejected alternative-3?
> > Store the cumulative size of remote tier log in-memory at
> RemoteLogManager
> > "*Cons*: Every time a broker starts-up, it will scan through all the
> > segments in the remote tier to initialise the in-memory value. This would
> > increase the broker start-up time."
> >
> > Keeping the source of truth to determine the remote-log-size in the
> leader
> > would be consistent across different implementations of the plugin. The
> > concern posted in the KIP is that we are calculating the remote-log-size
> on
> > each iteration of the cleaner thread (say 5 mins). If we calculate only
> > once during broker startup or during the leadership reassignment, do we
> > still need the cache?
> >
> > The broker startup-time won't be affected by the remote log manager
> > initialisation. The broker continue to start accepting the new
> > produce/fetch requests, while the RLM thread in the background can
> > determine the remote-log-size once and start copying/deleting the
> segments.
> >
> > Thanks,
> > Kamal
> >
> > On Thu, Jun 1, 2023 at 2:08 PM Divij Vaidya 
> > wrote:
> >
> > > Satish / Jun
> > >
> > > Do you have any thoughts on this?
> > >
> > > --
> > > Divij Vaidya
> > >
> > >
> > >
> > > On Tue, Feb 14, 2023 at 4:15 PM Divij Vaidya 
> > > wrote:
> > >
> > > > Hey Jun
> > > >
> > > > It has been a while since this KIP got some attention. While we wait
> > for
> > > > Satish to chime in here, perhaps I can answer your question.
> > > >
> > > > > Could you explain how you exposed the log size in your KIP-405
> > > > implementation?
> > > >
> > > > The APIs available in RLMM as per KIP405
> > > > are, addRemoteLogSegmentMetadata(), updateRemoteLogSegmentMetadata(),
> > > remoteLogSegmentMetadata(), highestOffsetForEpoch(),
> > > putRemotePartitionDeleteMetadata(), listRemoteLogSegments(),
> > > onPartitionLeadershipChanges()
> > > > and onStopPartitions(). None of these APIs allow us to expose the log
> > > size,
> > > > hence, the only option that remains is to list all segments using
> > > > listRemoteLogSegments() and aggregate them every time we require to
> > > > calculate the size. Based on our prior discussion, this requires
> > reading
> > > > all segment metadata which won't work for non-local RLMM
> > implementations.
> > > > Satish's implementation also performs a full scan and calculates the
> > > > aggregate. see:
> > > >
> > >
> >
> https://github.com/satishd/kafka/blob/2.8.x-tiered-storage/core/src/main/scala/kafka/log/remote/RemoteLogManager.scala#L619
> > > >
> > > >
> > > > Does this answer your question?
> > > >
> > > > --
> > > > Divij Vaidya
> > > >
> > > >
> > > >
> > > > On Tue, Dec 20, 2022 at 8:40 PM Jun Rao 
> > > wrote:
> > > >
> > > >> Hi, Divij,
> > > >>
> > > >> Thanks for the explanation.
> > > >>
> > > >> Good question.
> > > >>
> > > >> Hi, Satish,
> > > >>
> > > >> Could you explain how you exposed the log size in your KIP-405
> > > >> implementation?
> > > >>
> > > >> Thanks,
> > > >>
> > > >> Jun
> > > >>
> > > >> On Tue, Dec 20, 2022 at 4:59 AM Divij Vaidya <
> divijvaidy...@gmail.com
> > >
> > > >> wrote:
> > > >>
> > > >> > Hey Jun
> > > >> >
> > > >> > Yes, it is possible to maintain the log size in the cache (see
> > > rejected
> > > >> > alternative#3 in the KIP) but I did not understand how it is
> > possible
> > > to
> > > >> > retrieve it without the new API. The log size could be calculated
> on
> > > >> > startup by scanning through the segments (though 

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #1964

2023-06-30 Thread Apache Jenkins Server
See 


Changes:


--
[...truncated 389831 lines...]

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 133 > 
MirrorConnectorsIntegrationTransactionsTest > 
testReplicationWithEmptyPartition() PASSED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 133 > 
MirrorConnectorsIntegrationTransactionsTest > testReplicateSourceDefault() 
STARTED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 133 > 
MirrorConnectorsIntegrationTransactionsTest > testReplicateSourceDefault() 
PASSED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 133 > 
MirrorConnectorsIntegrationTransactionsTest > testOffsetSyncsTopicsOnTarget() 
STARTED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 131 > 
MirrorConnectorsIntegrationTransactionsTest > testReplicateSourceDefault() 
PASSED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 131 > 
MirrorConnectorsIntegrationTransactionsTest > testOffsetSyncsTopicsOnTarget() 
STARTED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 133 > 
MirrorConnectorsIntegrationTransactionsTest > testOffsetSyncsTopicsOnTarget() 
PASSED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 133 > 
MirrorConnectorsIntegrationTransactionsTest > testReplicateTargetDefault() 
STARTED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 131 > 
MirrorConnectorsIntegrationTransactionsTest > testOffsetSyncsTopicsOnTarget() 
PASSED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 131 > 
MirrorConnectorsIntegrationTransactionsTest > testReplicateTargetDefault() 
STARTED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 133 > 
MirrorConnectorsIntegrationTransactionsTest > testReplicateTargetDefault() 
PASSED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 133 > 
MirrorConnectorsIntegrationTransactionsTest > 
testOffsetTranslationBehindReplicationFlow() STARTED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 131 > 
MirrorConnectorsIntegrationTransactionsTest > testReplicateTargetDefault() 
PASSED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 131 > 
MirrorConnectorsIntegrationTransactionsTest > 
testOffsetTranslationBehindReplicationFlow() STARTED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 133 > 
MirrorConnectorsIntegrationTransactionsTest > 
testOffsetTranslationBehindReplicationFlow() PASSED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 133 > 
MirrorConnectorsIntegrationTransactionsTest > 
testNoCheckpointsIfNoRecordsAreMirrored() STARTED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 131 > 
MirrorConnectorsIntegrationTransactionsTest > 
testOffsetTranslationBehindReplicationFlow() PASSED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 131 > 
MirrorConnectorsIntegrationTransactionsTest > 
testNoCheckpointsIfNoRecordsAreMirrored() STARTED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 131 > 
MirrorConnectorsIntegrationTransactionsTest > 
testNoCheckpointsIfNoRecordsAreMirrored() PASSED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 131 > 
MirrorConnectorsIntegrationTransactionsTest > 
testOneWayReplicationWithAutoOffsetSync() STARTED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 133 > 
MirrorConnectorsIntegrationTransactionsTest > 
testNoCheckpointsIfNoRecordsAreMirrored() PASSED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 133 > 
MirrorConnectorsIntegrationTransactionsTest > 
testOneWayReplicationWithAutoOffsetSync() STARTED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 131 > 
MirrorConnectorsIntegrationTransactionsTest > 
testOneWayReplicationWithAutoOffsetSync() PASSED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 131 > 
MirrorConnectorsIntegrationTransactionsTest > testSyncTopicConfigs() STARTED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 133 > 
MirrorConnectorsIntegrationTransactionsTest > 
testOneWayReplicationWithAutoOffsetSync() PASSED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 133 > 
MirrorConnectorsIntegrationTransactionsTest > testSyncTopicConfigs() STARTED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 131 > 
MirrorConnectorsIntegrationTransactionsTest > testSyncTopicConfigs() PASSED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 131 > 
MirrorConnectorsIntegrationTransactionsTest > testRestartReplication() STARTED

Gradle Test Run :connect:mirror:integrationTest > Gradle Test Executor 133 > 

Re: Apache Kafka 3.6.0 release

2023-06-30 Thread Christo Lolov
Hello!

I will add KIP-902 to the release plan. I would appreciate a few more
reviews on the pull request (https://github.com/apache/kafka/pull/13260)
for that KIP as the longer we have it in trunk with tests running against
it the more confidence we will have before the release.

Best,
Christo

On Sat, 24 Jun 2023 at 17:14, Chris Egerton  wrote:

> Thanks Satish!
>
> On Sat, Jun 24, 2023 at 7:34 AM Satish Duggana 
> wrote:
>
> > Thanks Chris for the update. I added KIP-875 to the 3.6.0 release plan
> > wiki. Please feel free to update it.
> >
> > ~Satish.
> >
> > On Fri, 23 Jun 2023 at 23:10, Chris Egerton 
> > wrote:
> > >
> > > Hi Satish,
> > >
> > > Could we add KIP-875 [1] to the release plan? It was partially released
> > in
> > > 3.5.0 and mentioned in the release plan [2], and since the rest (APIs
> to
> > > reset and alter offsets for connectors) has now been merged to trunk,
> we
> > > can let people know that the remainder should be available in the next
> > > release.
> > >
> > > [1] -
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-875%3A+First-class+offsets+support+in+Kafka+Connect
> > > [2] -
> > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+3.5.0
> > >
> > > Cheers,
> > >
> > > Chris
> > >
> > > On Tue, Jun 20, 2023 at 12:10 AM Satish Duggana <
> > satish.dugg...@gmail.com>
> > > wrote:
> > >
> > > > Thanks Ivan for the update. Added KIP-917 to 3.6.0 Release Plan wiki.
> > > > Please feel free to update the status in the wiki.
> > > >
> > > > On Mon, 19 Jun 2023 at 18:35, Ivan Yurchenko <
> ivan0yurche...@gmail.com
> > >
> > > > wrote:
> > > > >
> > > > > Thank you. If by closing you mean summing up and announcing the
> > result,
> > > > > then already did.
> > > > >
> > > > > Ivan
> > > > >
> > > > >
> > > > > On Mon, 19 Jun 2023 at 15:28, Satish Duggana <
> > satish.dugg...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi Ivan,
> > > > > > Sure, KIP freeze date is 26th July 23 for 3.6.0. Please close the
> > > > > > voting for KIP acceptance before that.
> > > > > >
> > > > > > Thanks,
> > > > > > Satish.
> > > > > >
> > > > > > On Mon, 19 Jun 2023 at 16:03, Ivan Yurchenko <
> > ivan0yurche...@gmail.com
> > > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > Hi,
> > > > > > >
> > > > > > > I would like to propose to include the newly accepted "KIP-917:
> > > > > > Additional
> > > > > > > custom metadata for remote log segment" [1] in the release
> plan.
> > > > Would it
> > > > > > > be possible?
> > > > > > > Thanks!
> > > > > > >
> > > > > > > Best,
> > > > > > > Ivan
> > > > > > >
> > > > > > > [1]
> > > > > > >
> > > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-917%3A+Additional+custom+metadata+for+remote+log+segment
> > > > > > >
> > > > > > > On Mon, 12 Jun 2023 at 13:22, Satish Duggana <
> > > > satish.dugg...@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hi,
> > > > > > > > I have created a release plan for Apache Kafka version 3.6.0
> > on the
> > > > > > > > wiki. You can access the release plan and all related
> > information
> > > > by
> > > > > > > > following this link:
> > > > > > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+3.6.0
> > > > > > > >
> > > > > > > > The release plan outlines the key milestones and important
> > dates
> > > > for
> > > > > > > > version 3.6.0. Currently, the following dates have been set
> > for the
> > > > > > > > release:
> > > > > > > >
> > > > > > > > KIP Freeze: 26th July 23
> > > > > > > > Feature Freeze : 16th Aug 23
> > > > > > > > Code Freeze : 30th Aug 23
> > > > > > > >
> > > > > > > > Please review the release plan and provide any additional
> > > > information
> > > > > > > > or updates regarding KIPs targeting version 3.6.0. If you
> have
> > > > > > > > authored any KIPs that are missing a status or if there are
> > > > incorrect
> > > > > > > > status details, please make the necessary updates and inform
> > me so
> > > > > > > > that I can keep the plan accurate and up to date.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Satish.
> > > > > > > >
> > > > > > > > On Mon, 17 Apr 2023 at 21:17, Luke Chen 
> > wrote:
> > > > > > > > >
> > > > > > > > > Thanks for volunteering!
> > > > > > > > >
> > > > > > > > > +1
> > > > > > > > >
> > > > > > > > > Luke
> > > > > > > > >
> > > > > > > > > On Mon, Apr 17, 2023 at 2:03 AM Ismael Juma <
> > ism...@juma.me.uk>
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Thanks for volunteering Satish. +1.
> > > > > > > > > >
> > > > > > > > > > Ismael
> > > > > > > > > >
> > > > > > > > > > On Sun, Apr 16, 2023 at 10:08 AM Satish Duggana <
> > > > > > > > satish.dugg...@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi,
> > > > > > > > > > > I would like to volunteer as release manager for the
> next
> > > > > > release,
> > > > > > > > > > > which will be Apache Kafka 3.6.0.
> > > > > > > > > > >
> > > > > > > > > > > 

[jira] [Created] (KAFKA-15137) Don't log the entire request in KRaftControllerChannelManager

2023-06-30 Thread David Arthur (Jira)
David Arthur created KAFKA-15137:


 Summary: Don't log the entire request in 
KRaftControllerChannelManager
 Key: KAFKA-15137
 URL: https://issues.apache.org/jira/browse/KAFKA-15137
 Project: Kafka
  Issue Type: Bug
Affects Versions: 3.5.0, 3.6.0
Reporter: David Arthur
Assignee: Alyssa Huang
 Fix For: 3.5.1


While debugging some junit tests, I noticed some really long log lines in 
KRaftControllerChannelManager. When the broker is down, we log a WARN that 
includes the entire UpdateMetadataRequest or LeaderAndIsrRequest. For large 
clusters, these can be really large requests, so this could potentially cause 
excessive output in the log4j logs.



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


Re: [DISCUSS] KIP-940: Broker extension point for validating record contents at produce time

2023-06-30 Thread Jorge Esteban Quilcate Otoya
Thank you both for the replies! A couple more comments:

On Tue, 27 Jun 2023 at 14:57, Edoardo Comar  wrote:

> Hi Jorge
> thanks for the feedback. Comments inline below
>
> > 1. Similar to Kirk's first point, I'm also concerned on how would the
> > plugin developers / operators be able to apply multiple policies and how
> > configurations would be passed to each policy.
>
> We’ve only attempted to tackle the “one plugin per broker” model with
> this KIP, as that’s the use-case we most clearly understand. Although,
> as noted in the rejected alternatives section, it would be possible to
> use a facade-like pattern to delegate from one plugin implementation
> to others. The reason we’ve avoided tackling multiple plugins is that
> it introduces further complexity (which takes precedence? Is
> configuration needed to say which plugin applies to which topic?
> Etc.), and we are concerned that without a clear use-case we might
> make decisions we later come to regret. Hopefully by offering minimal
> configuration options, we don’t hinder a future “support multiple
> record validation policies” KIP.
>

Got it. Thanks!


>
> > Some approaches from other plugins we can get some inspiration from:
> >
> > - AlterConfig, CreateTopic policies are designed to be 1 policy
> > implementing the different APIs. Up to the plugin developer to pull
> > policies together and configure it on the broker side. I guess for Record
> > Validation this may be cumbersome considering some Schema Registry
> > providers may want to offer implementations for their own backend.
> >
> > - Connect Transforms: here there's a named set of plugins to apply per
> > connector, and each transform has its own configuration defined by
> prefix.
> > Personally, I'd consider this one an interesting approach if we decide to
> > allow multiple record validations to be configured.
> >
> > - Tiered Storage (probably Connectors as well) have class-loader aware
> > implementations with class path specific to the plugin. Not sure if this
> is
> > something to discuss on the KIP or later on the PR, but we could mention
> > something on how this plugin would deal with dependency conflicts (e.g.
> > different jackson version between broker, plugin(s)).
>
>
> Thanks for highlighting all of these places where we can draw
> inspiration. We’ve updated the KIP with an additional classloader
> property to match the tiered storage implementation. It seems likely
> that record validation policy implementations will live in the
> codebase of their corresponding schema registry (as is the case,
> today, for the client serdes used to integrate with a schema registry)
> - so it makes sense to insulate their implementation from specific
> .jar versions that may (or may not) be present in a particular version
> of the broker.
>
> > Also, by potentially supporting multiple plugins for record validation,
> it
> > would be important to consider if it's an all or nothing relation, or
> > posible to choose _some_ policies apply per topic.
> > I see there's some preference for setting the validation policy name on
> the
> > topic, though this could be cumbersome to operate: topic creation users
> may
> > not be aware of the record validation (similar to CreateTopic/AlterConfig
> > policies) and would impose additional coordination.
> > Maybe a flag to whether apply policies or not would be a better approach?
>
> Could you elaborate more on your comments about “maybe a flag to
> whether to apply policies or not would be a better approach?”. We
> thought that setting the ‘record.validation.policy’ property on a
> topic to a value supported by the plugin was such a flag - but it
> sounds like you might have a different approach in mind?
>
>
The current proposal is to have ‘record.validation.policy’ per topic
(default null). A flag would be something like
‘record.validation.policy.enable’ (default=false) may be simpler to
configure from the user perspective.

Also, at the moment, is a bit unclear to me what value the topic config
‘record.validation.policy’ should contain: is the policy class name? How is
the policy expected to use the name received?


> > 2. Have you consider adding the record metadata to the API? It may be
> > useful for logging purposes (e.g. if record validation fails, to log
> > topic-partition), or some policies are interested on record metadata
> (e.g.
> > compression, timestamp type, etc.)
>
> The topic/partition is available to the plugin via the TopicMetadata
> interace. Additional record properties could be added to the
> ‘RecordProxy’ interface, however the topic of how rich to make the
> interface was a sticking point for KIP-729. The intent behind the
> ‘TopicMetadata’ and ‘RecordProxy’ classes is that they can be extended
> in the future without breaking existing plugin implementations - so
> we’re not precluding further properties from being added if we’ve been
> too austere.
>

I see, agree.


>
> > 3. A minor comment for consistency regarding the 

Jenkins build is unstable: Kafka » Kafka Branch Builder » trunk #1963

2023-06-30 Thread Apache Jenkins Server
See 




[jira] [Created] (KAFKA-15136) The inspection of field allowAutoTopicCreation field in MetadataRequest is unreasonable

2023-06-30 Thread Xiaobing Fang (Jira)
Xiaobing Fang created KAFKA-15136:
-

 Summary: The inspection of field allowAutoTopicCreation field in 
MetadataRequest is unreasonable
 Key: KAFKA-15136
 URL: https://issues.apache.org/jira/browse/KAFKA-15136
 Project: Kafka
  Issue Type: Improvement
  Components: clients
Reporter: Xiaobing Fang


```java

if (!data.allowAutoTopicCreation() && version < 4)
throw new UnsupportedVersionException("MetadataRequest versions older than 4 
don't support the " +
"allowAutoTopicCreation field");

```

Background:
Based on my understanding, the code in MetadataRequest is intended to ignore 
the allowAutoTopicCreation field when version≤4. However, if kafka server is 
configured with "auto.create.topics.enable"=false and the client sets 
allowAutoTopicCreation=false, UnsupportedVersionExceptionwill be thrown, which 
is not expected.

Issues:
 # `KafkaAdminClient#handleDescribeTopicsByNames()` throws an exception when 
sending MetadataRequest to a lower version server, making it unusable.
 # MetadataRequestTest avoids this issue by setting 
`allowAutoTopicCreation=true` in tests for version≤4, but this is not 
reasonable. And the comments in 
[testAutoTopicCreation|https://github.com/apache/kafka/blob/1f4cbc5d53259031123b6e9e6bb9a5bbe1e084e8/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala#L146]
 may also be problematic.

Solution:  
 # Remove the checking code in MetadataRequest.
 # Add a field `hasSetAllowAutoTopicCreation` in MetadataRequest. Only throw an 
exception when `version≤4` and `hasSetAllowAutoTopicCreation=true`, without 
considering the value of allowAutoTopicCreation field.

 

If there is indeed an issue, I can work on fixing it. Looking forward to your 
reply.



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


[jira] [Created] (KAFKA-15135) RLM listener configurations passed but ignored by RLMM

2023-06-30 Thread Jorge Esteban Quilcate Otoya (Jira)
Jorge Esteban Quilcate Otoya created KAFKA-15135:


 Summary: RLM listener configurations passed but ignored by RLMM
 Key: KAFKA-15135
 URL: https://issues.apache.org/jira/browse/KAFKA-15135
 Project: Kafka
  Issue Type: Bug
  Components: core
Reporter: Jorge Esteban Quilcate Otoya


As describe here [1] properties captured from listener are passed but ignored 
by TopicBasedRLMM.

 

[1] https://github.com/apache/kafka/pull/13828#issuecomment-1611155345



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


Re: Requesting permissions to contribute to Apache Kafka

2023-06-30 Thread Mayank Shekhar Narula
thanks Divij!

On Thu, Jun 29, 2023 at 8:19 PM Divij Vaidya 
wrote:

> You should be all set.
>
> --
> Divij Vaidya
>
>
>
> On Thu, Jun 29, 2023 at 8:45 PM Mayank Shekhar Narula <
> mayanks.nar...@gmail.com> wrote:
>
> >  - can someone grant these? Thanks!
> >
> > As requested here -
> >
> >
> https://cwiki.apache.org/confluence/display/kafka/kafka+improvement+proposals#KafkaImprovementProposals-GettingStarted
> > "
> >
> > Wiki Id - mayanks*.*narula
> > Jira  Id - mayanksnarula
> >
> > Notice that Jira Id doesn't have the ".", whereas Wiki id does have ".".
> >
> >
> >
> > --
> > Regards,
> > Mayank Shekhar Narula
> >
>


-- 
Regards,
Mayank Shekhar Narula