Re: [DISCUSS] KIP-1014: Managing Unstable Metadata Versions in Apache Kafka

2024-01-15 Thread Colin McCabe
On Fri, Jan 12, 2024, at 11:32, Artem Livshits wrote:
> I think using feature flags (whether we support a framework and tooling for
> feature flags or just an ad-hoc XyzEnabled flag) can be an alternative to
> this KIP.  I think the value of this KIP is that it's trying to propose a
> systemic approach for gating functionality that may take multiple releases
> to develop.  A problem with ad-hoc feature flags is that it's useful during
> feature development, so that people who are working on this feature (or are
> interested in beta-testing the feature) can get early access (without any
> guarantees on compatibility or even correctness); but then the feature
> flags often stick forever after the feature development is done (and as
> time moves one, the new code is written in such a way that it's not
> possible to turn the feature off any more cleanly).
>

Hi Artem,

I think feature flags are somewhat orthogonal to the stable / unstable 
discussion. Even if every new feature was a feature flag, you probably still 
wouldn't want to stabilize the features immediately, to avoid maintaining a lot 
of alpha stuff forever.

(I also think that feature flags should be used sparingly, if at all, because 
of the way that they exponentially increase the test matrix. But that's a 
tangent, I think, given the first point...)

>
> I'd also clarify how I think about "stable".  Ismael made a comment "
> something is stable in the "this is battle-tested" sense.".  I don't think
> it has to be "battle-tested", it just has to meet the bar of being in the
> trunk.  Again, thinking of a small single-commit feature -- to commit to
> trunk, the feature doesn't have to be "battle-tested", but it should be
> complete (and not just a bunch of TODOs), with tests written and some level
> of dev-testing done, so that once the release is cut, we can find and fix
> bugs and make it release-quality (as opposed to reverting the whole
> thing).  We can apply the same "stability" bar for features to be in the
> stable MV -- fully complete, tests written and some level of dev testing
> done.
>

I'm struggling a bit with your phrasing. Are you suggesting that unstable MVs 
would not be able to be in trunk? I think we do want them to be able to go into 
trunk. If they have to go into a branch, then there is actually no need for any 
of this.

Doing big features in long-lived branches is one genuine alternative to 
unstable MVs, I think. But it's not an alternative that works well with our 
current tooling for continuous integration, deployment, building, etc. I think 
it would also impact developer productivity somewhat negatively.

best,
Colin


>
> -Artem
>
> On Fri, Jan 12, 2024 at 10:12 AM Justine Olshan
>  wrote:
>
>> Hi Ismael,
>>
>> I debated including something about feature flags in my last comment, but
>> maybe I should have.
>> What you said makes sense.
>>
>> Justine
>>
>> On Fri, Jan 12, 2024 at 9:31 AM Ismael Juma  wrote:
>>
>> > Justine,
>> >
>> > For features that are not production-ready, they should have an
>> additional
>> > configuration (not the metadata version) that enables/disables it. The MV
>> > specific features we ship are something we have to support and make sure
>> we
>> > don't break going forward.
>> >
>> > Ismael
>> >
>> > On Fri, Jan 12, 2024 at 9:26 AM Justine Olshan
>> > 
>> > wrote:
>> >
>> > > Hi Ismael,
>> > >
>> > > I think the concern I have about a MV for a feature that is not
>> > production
>> > > ready is that it blocks any development/features with higher MV
>> versions
>> > > that could be production ready.
>> > >
>> > > I do see your point though. Previously MV/IBP was about pure broker
>> > > compatibility and not about the confidence in the feature it is
>> gating. I
>> > > do wonder though if it could be useful to have that sort of gating.
>> > > I also wonder if an internal config could be useful so that the average
>> > > user doesn't have to worry about the complexities of unstable metadata
>> > > versions (and their risks).
>> > >
>> > > I am ok with options 2 and 2 as well by the way.
>> > >
>> > > Justine
>> > >
>> > > On Fri, Jan 12, 2024 at 7:36 AM Ismael Juma  wrote:
>> > >
>> > > > Hi,
>> > > >
>> > > > Thanks for the KIP.
>> > > >
>> > > > Reading the discussion, I think a lot of the confusion is due to the
>> > > > "unstable" naming. People are then trying to figure out when we think
>> > > > something is stable in the "this is battle-tested" sense. But this
>> flag
>> > > > should not be about that. We can have an MV for a feature that is not
>> > yet
>> > > > production-ready (and we did that when KRaft itself was not
>> production
>> > > > ready). I think this flag is about metadata versions that are
>> basically
>> > > > unsupported - if you use it, you get to keep the pieces. They exist
>> > > solely
>> > > > to make the lives of Apache Kafka developers easier. I would even
>> > suggest
>> > > > that the config we introduce be of the internal variety, ie it won't
>> > show
>> 

[jira] [Resolved] (KAFKA-16121) Partition reassignments in ZK migration dual write mode stalled until leader epoch incremented

2024-01-15 Thread Colin McCabe (Jira)


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

Colin McCabe resolved KAFKA-16121.
--
Fix Version/s: 3.7.0
 Reviewer: Colin McCabe
 Assignee: David Mao
   Resolution: Duplicate

> Partition reassignments in ZK migration dual write mode stalled until leader 
> epoch incremented
> --
>
> Key: KAFKA-16121
> URL: https://issues.apache.org/jira/browse/KAFKA-16121
> Project: Kafka
>  Issue Type: Bug
>Reporter: David Mao
>Assignee: David Mao
>Priority: Major
> Fix For: 3.7.0
>
>
> I noticed this in an integration test in 
> https://github.com/apache/kafka/pull/15184
> In ZK mode, partition leaders rely on the LeaderAndIsr request to be notified 
> of new replicas as part of a reassignment. In ZK mode, we ignore any 
> LeaderAndIsr request where the partition leader epoch is less than or equal 
> to the current partition leader epoch.
> In KRaft mode, we do not bump the leader epoch when starting a new 
> reassignment, see: `triggerLeaderEpochBumpIfNeeded`. This means that the 
> leader will ignore the LISR request initiating the reassignment until a 
> leader epoch bump is triggered through another means, for instance preferred 
> leader election.



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


[jira] [Created] (KAFKA-16144) Controller leader checkQuorum timer should skip only 1 controller case

2024-01-15 Thread Luke Chen (Jira)
Luke Chen created KAFKA-16144:
-

 Summary: Controller leader checkQuorum timer should skip only 1 
controller case
 Key: KAFKA-16144
 URL: https://issues.apache.org/jira/browse/KAFKA-16144
 Project: Kafka
  Issue Type: Bug
Reporter: Luke Chen


In KAFKA-15489, we fixed the potential "split brain" issue by adding the check 
quorum timer. This timer will be updated when the follower fetch request 
arrived. And it expires the timer when the there are no majority of voter 
followers fetch from leader, and resign the leadership. 

But in KAFKA-15489, we forgot to consider the case where there's only 1 
controller node. If there's only 1 controller node (and no broker node), there 
will be no fetch request arrived, so the timer will expire each time. However, 
if there's only 1 node, we don't have to care about the "check quorum" at all. 
We should skip the check for only 1 controller node case.

Currently, this issue will happen only when there's 1 controller node and no 
any broker node (i.e. no fetch request sent to the controller). 



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


[jira] [Resolved] (KAFKA-15809) Update broker's metadata schema to include TS enable status

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


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

Phuc Hong Tran resolved KAFKA-15809.

Resolution: Won't Fix

> Update broker's metadata schema to include TS enable status
> ---
>
> Key: KAFKA-15809
> URL: https://issues.apache.org/jira/browse/KAFKA-15809
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Phuc Hong Tran
>Assignee: Phuc Hong Tran
>Priority: Minor
>  Labels: KIP-405
> Fix For: 3.8.0
>
>
> Currently controller doesn't have the visibility of all brokers's TS enable 
> status. As mentioned in KAFKA-15341, we need to add metadata about TS enable 
> status of brokers so that controller can check for these status before 
> enabling TS per topic



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


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

2024-01-15 Thread Luke Chen
+1 binding from me.

Thanks for the KIP!
Luke

On Fri, Jan 12, 2024 at 5:41 PM Federico Valeri 
wrote:

> +1 non binding
>
> Thanks
>
> On Fri, Jan 12, 2024 at 1:31 AM Boudjelda Mohamed Said
>  wrote:
> >
> > +1 (binding)
> >
> >
> > On Fri, Jan 12, 2024 at 1:21 AM Satish Duggana  >
> > wrote:
> >
> > > +1 (binding)
> > >
> > > Thanks,
> > > Satish.
> > >
> > > On Thu, 11 Jan 2024 at 17:52, Divij Vaidya 
> > > wrote:
> > > >
> > > > +1 (binding)
> > > >
> > > > Divij Vaidya
> > > >
> > > >
> > > >
> > > > On Tue, Dec 26, 2023 at 7:05 AM Kamal Chandraprakash <
> > > > kamal.chandraprak...@gmail.com> wrote:
> > > >
> > > > > +1 (non-binding). Thanks for the KIP!
> > > > >
> > > > > --
> > > > > Kamal
> > > > >
> > > > > On Thu, Dec 21, 2023 at 2:23 PM Christo Lolov <
> christolo...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Heya all!
> > > > > >
> > > > > > KIP-1005 (
> > > > > >
> > > > > >
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1005%3A+Expose+EarliestLocalOffset+and+TieredOffset
> > > > > > )
> > > > > > has been open for around a month with no further comments - I
> would
> > > like
> > > > > to
> > > > > > start a voting round on it!
> > > > > >
> > > > > > Best,
> > > > > > Christo
> > > > > >
> > > > >
> > >
>


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

2024-01-15 Thread Apache Jenkins Server
See 




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

2024-01-15 Thread Apache Jenkins Server
See 




[jira] [Created] (KAFKA-16142) Update metrics documentation for errors and new metrics

2024-01-15 Thread Kirk True (Jira)
Kirk True created KAFKA-16142:
-

 Summary: Update metrics documentation for errors and new metrics
 Key: KAFKA-16142
 URL: https://issues.apache.org/jira/browse/KAFKA-16142
 Project: Kafka
  Issue Type: Task
  Components: clients, consumer
Affects Versions: 3.7.0
Reporter: Kirk True
 Fix For: 3.8.0






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


[jira] [Created] (KAFKA-16143) New metrics for KIP-848 protocol

2024-01-15 Thread Kirk True (Jira)
Kirk True created KAFKA-16143:
-

 Summary: New metrics for KIP-848 protocol
 Key: KAFKA-16143
 URL: https://issues.apache.org/jira/browse/KAFKA-16143
 Project: Kafka
  Issue Type: Task
  Components: clients, consumer
Affects Versions: 3.7.0
Reporter: Kirk True
Assignee: Philip Nee
 Fix For: 3.8.0






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


Re: [DISCUSS] KIP-971 Expose replication-offset-lag MirrorMaker2 metric

2024-01-15 Thread Elxan Eminov
Apologies, forgot to reply on your last comment about the metric name.
I believe both replication-lag and record-lag are a little too abstract -
what do you think about either leaving it as replication-offset-lag or
renaming to replication-record-lag?

Thanks

On Wed, 10 Jan 2024 at 15:31, Mickael Maison 
wrote:

> Hi Elxan,
>
> Thanks for the KIP, it looks like a useful addition.
>
> Can you add to the KIP the default value you propose for
> replication.lag.metric.refresh.interval? In MirrorMaker most interval
> configs can be set to -1 to disable them, will it be the case for this
> new feature or will this setting only accept positive values?
> I also wonder if replication-lag, or record-lag would be clearer names
> instead of replication-offset-lag, WDYT?
>
> Thanks,
> Mickael
>
> On Wed, Jan 3, 2024 at 6:15 PM Elxan Eminov 
> wrote:
> >
> > Hi all,
> > Here is the vote thread:
> > https://lists.apache.org/thread/ftlnolcrh858dry89sjg06mdcdj9mrqv
> >
> > Cheers!
> >
> > On Wed, 27 Dec 2023 at 11:23, Elxan Eminov 
> wrote:
> >
> > > Hi all,
> > > I've updated the KIP with the details we discussed in this thread.
> > > I'll call in a vote after the holidays if everything looks good.
> > > Thanks!
> > >
> > > On Sat, 26 Aug 2023 at 15:49, Elxan Eminov 
> > > wrote:
> > >
> > >> Relatively minor change with a new metric for MM2
> > >>
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-971%3A+Expose+replication-offset-lag+MirrorMaker2+metric
> > >>
> > >
>


Re: [DISCUSS] KIP-971 Expose replication-offset-lag MirrorMaker2 metric

2024-01-15 Thread Elxan Eminov
Hi Mickael,
Thanks for the response.
I've added the proposed default value for the refresh interval config.
I've also added a new config to the KIP that will enable a TTL-based
eviction policy for the in-memory LRO cache, as @huqedi suggested - also
added a proposed default for this configuration.
I did not get a chance/could not figure out a way to verify how much sense
these default values make - so I'd be happy to be challenged on those.
I've also skimmed through the MirrorMaker configs to check usages for
configs that accept `-1` values - if my understanding is correct, `-1` is
in the case of this metric would be intended to indicate that this metric
should not be reported, since it would not be possible without the periodic
LEO poll job.
If my understanding is correct, I believe it makes sense to allow this -
I've added a sentence about this to the KIP.
Please take a look and let me know what you think,

Thanks!

On Wed, 10 Jan 2024 at 15:31, Mickael Maison 
wrote:

> Hi Elxan,
>
> Thanks for the KIP, it looks like a useful addition.
>
> Can you add to the KIP the default value you propose for
> replication.lag.metric.refresh.interval? In MirrorMaker most interval
> configs can be set to -1 to disable them, will it be the case for this
> new feature or will this setting only accept positive values?
> I also wonder if replication-lag, or record-lag would be clearer names
> instead of replication-offset-lag, WDYT?
>
> Thanks,
> Mickael
>
> On Wed, Jan 3, 2024 at 6:15 PM Elxan Eminov 
> wrote:
> >
> > Hi all,
> > Here is the vote thread:
> > https://lists.apache.org/thread/ftlnolcrh858dry89sjg06mdcdj9mrqv
> >
> > Cheers!
> >
> > On Wed, 27 Dec 2023 at 11:23, Elxan Eminov 
> wrote:
> >
> > > Hi all,
> > > I've updated the KIP with the details we discussed in this thread.
> > > I'll call in a vote after the holidays if everything looks good.
> > > Thanks!
> > >
> > > On Sat, 26 Aug 2023 at 15:49, Elxan Eminov 
> > > wrote:
> > >
> > >> Relatively minor change with a new metric for MM2
> > >>
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-971%3A+Expose+replication-offset-lag+MirrorMaker2+metric
> > >>
> > >
>


Re: [DISCUSS] KIP-981: Manage Connect topics with custom implementation of Admin

2024-01-15 Thread Omnia Ibrahim
Hi It has been a while! but can I have a feedback on this. It would be nice
to unify this between MM2 and Connect as well.

Thanks

On Thu, Oct 19, 2023 at 3:14 PM Omnia Ibrahim 
wrote:

> Hi, any thoughts on this kip?
>
> Thanks
>
> On Tue, Sep 19, 2023 at 6:04 PM Omnia Ibrahim 
> wrote:
>
>> Hi everyone,
>> I want to start the discussion of the KIP-981 to extend Connect to use
>> org.apache.kafka.clients.admin.ForwardingAdminClient instead of
>> KafkaAdminClient 
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-981%3A+Manage+Connect+topics+with+custom+implementation+of+Admin
>>
>>
>> Thanks for your time and feedback
>> Omnia
>>
>


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

2024-01-15 Thread Apache Jenkins Server
See 




Re: [VOTE] 3.7.0 RC2

2024-01-15 Thread Kirk True
Hi Stanislav,

On Sun, Jan 14, 2024, at 1:17 PM, Stanislav Kozlovski wrote:
> Hey Kirk and Chris,
> 
> Unless I'm missing something - KAFKALESS-16029 is simply a bad log due to
> improper closing. And the PR description implies this has been present
> since 3.5. While annoying, I don't see a strong reason for this to block
> the release.

I would imagine that it would result in concerned users reporting the issue.

I took another look, and the code that causes the issue was indeed changed in 
3.7. It is easily reproducible.

The PR is ready for review: https://github.com/apache/kafka/pull/15186

Thanks,
Kirk

Re: [VOTE] 3.7.0 RC2

2024-01-15 Thread Stanislav Kozlovski
I wanted to circle back and confirm the integration tests + system tests,
plus give an overall update regarding status.

The integration tests have a fair amount of flakes. I ran and inspected 3
consecutive builds (57
, 58
, 59
), then
cross-checked each run's failures via a script of mine to see any
consistent failures.

Three tests proved very flaky. Two are related to KIP-848 running under
KRaft. The third one is a Trogdor test. All 3 tests pass locally, hence I
deem them not blockers for the release. Especially since KIP-848 is in
early access, I am not particularly concerned with a flaky test. I opened
three JIRAs to track them:
- https://issues.apache.org/jira/browse/KAFKA-16134
- https://issues.apache.org/jira/browse/KAFKA-16135
- https://issues.apache.org/jira/browse/KAFKA-16136

As for the system tests, I again ran 2 consecutive builds (1
,
2
)
and I found 4 tests that exhibit consecutive failures.
- The whole analysis: https://hackmd.io/@hOneAGCrSmKSpL8VF-1HWQ/HyRgRJmta

The failing tests:
StreamsStandbyTask - https://issues.apache.org/jira/browse/KAFKA-16141
StreamsUpgradeTest - https://issues.apache.org/jira/browse/KAFKA-16139
QuotaTest - https://issues.apache.org/jira/browse/KAFKA-16138
ZookeeperMigrationTest - https://issues.apache.org/jira/browse/KAFKA-16140

I am reaching out to subject matter experts regarding the failures.

Thanks to everyone who contributed in testing the release. Here is a
general update regarding known blockers that were recently found:

We are treating https://issues.apache.org/jira/browse/KAFKA-16131 and
https://issues.apache.org/jira/browse/KAFKA-16101 as blockers.

https://issues.apache.org/jira/browse/KAFKA-16132 is a potential other
issue that will likely be treated as a blocker

Best,
Stanislav

On Mon, Jan 15, 2024 at 12:04 PM Jakub Scholz  wrote:

> *> Hi Jakub,> > Thanks for trying the RC. I think what you found is a
> blocker bug because it *
> *> will generate huge amount of logspam. I guess we didn't find it in junit
> tests *
> *> since logspam doesn't fail the automated tests. But certainly it's not
> suitable *
> *> for production. Did you file a JIRA yet?*
>
> Hi Colin,
>
> I opened https://issues.apache.org/jira/browse/KAFKA-16131.
>
> Thanks & Regards
> Jakub
>
> On Mon, Jan 15, 2024 at 8:57 AM Colin McCabe  wrote:
>
> > Hi Stanislav,
> >
> > Thanks for making the first RC. The fact that it's titled RC2 is messing
> > with my mind a bit. I hope this doesn't make people think that we're
> > farther along than we are, heh.
> >
> > On Sun, Jan 14, 2024, at 13:54, Jakub Scholz wrote:
> > > *> Nice catch! It does seem like we should have gated this behind the
> > > metadata> version as KIP-858 implies. Is the cluster configured with
> > > multiple log> dirs? What is the impact of the error messages?*
> > >
> > > I did not observe any obvious impact. I was able to send and receive
> > > messages as normally. But to be honest, I have no idea what else
> > > this might impact, so I did not try anything special.
> > >
> > > I think everyone upgrading an existing KRaft cluster will go through
> this
> > > stage (running Kafka 3.7 with an older metadata version for at least a
> > > while). So even if it is just a logged exception without any other
> > impact I
> > > wonder if it might scare users from upgrading. But I leave it to others
> > to
> > > decide if this is a blocker or not.
> > >
> >
> > Hi Jakub,
> >
> > Thanks for trying the RC. I think what you found is a blocker bug because
> > it will generate huge amount of logspam. I guess we didn't find it in
> junit
> > tests since logspam doesn't fail the automated tests. But certainly it's
> > not suitable for production. Did you file a JIRA yet?
> >
> > > On Sun, Jan 14, 2024 at 10:17 PM Stanislav Kozlovski
> > >  wrote:
> > >
> > >> Hey Luke,
> > >>
> > >> This is an interesting problem. Given the fact that the KIP for
> having a
> > >> 3.8 release passed, I think it weights the scale towards not calling
> > this a
> > >> blocker and expecting it to be solved in 3.7.1.
> > >>
> > >> It is unfortunate that it would not seem safe to migrate to KRaft in
> > 3.7.0
> > >> (given the inability to rollback safely), but if that's true - the
> same
> > >> case would apply for 3.6.0. So in any case users w\ould be expected to
> > use a
> > >> patch release for this.
> >
> > Hi Luke,
> >
> > Thanks for testing rollback. I think this 

[jira] [Created] (KAFKA-16141) StreamsStandbyTask##test_standby_tasks_rebalanceArguments:{ “metadata_quorum”: “ISOLATED_KRAFT”, “use_new_coordinator”: false} fails consistently in 3.7

2024-01-15 Thread Stanislav Kozlovski (Jira)
Stanislav Kozlovski created KAFKA-16141:
---

 Summary: 
StreamsStandbyTask##test_standby_tasks_rebalanceArguments:{ “metadata_quorum”: 
“ISOLATED_KRAFT”, “use_new_coordinator”: false} fails consistently in 3.7
 Key: KAFKA-16141
 URL: https://issues.apache.org/jira/browse/KAFKA-16141
 Project: Kafka
  Issue Type: Test
Affects Versions: 3.7.0
Reporter: Stanislav Kozlovski


{code:java}
kafkatest.tests.streams.streams_standby_replica_test.StreamsStandbyTask#test_standby_tasks_rebalanceArguments:{
 “metadata_quorum”: “ISOLATED_KRAFT”, “use_new_coordinator”: false}

TimeoutError("Did expect to read 'ACTIVE_TASKS:2 STANDBY_TASKS:[1-3]' from 
ubuntu@worker26")
Traceback (most recent call last):
  File 
"/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/venv/lib/python3.7/site-packages/ducktape/tests/runner_client.py",
 line 184, in _do_run
data = self.run_test()
  File 
"/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/venv/lib/python3.7/site-packages/ducktape/tests/runner_client.py",
 line 262, in run_test
return self.test_context.function(self.test)
  File 
"/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/venv/lib/python3.7/site-packages/ducktape/mark/_mark.py",
 line 433, in wrapper
return functools.partial(f, *args, **kwargs)(*w_args, **w_kwargs)
  File 
"/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/tests/kafkatest/tests/streams/streams_standby_replica_test.py",
 line 79, in test_standby_tasks_rebalance
self.wait_for_verification(processor_1, "ACTIVE_TASKS:2 
STANDBY_TASKS:[1-3]", processor_1.STDOUT_FILE)
  File 
"/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/tests/kafkatest/tests/streams/base_streams_test.py",
 line 96, in wait_for_verification
err_msg="Did expect to read '%s' from %s" % (message, 
processor.node.account))
  File 
"/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/venv/lib/python3.7/site-packages/ducktape/utils/util.py",
 line 58, in wait_until
raise TimeoutError(err_msg() if callable(err_msg) else err_msg) from 
last_exception
ducktape.errors.TimeoutError: Did expect to read 'ACTIVE_TASKS:2 
STANDBY_TASKS:[1-3]' from ubuntu@worker26
 {code}
 



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


[jira] [Created] (KAFKA-16140) zookeeper_migration_test#TestMigration#test_recooncile_kraft_to_zk system test fails concistently on 3.7

2024-01-15 Thread Stanislav Kozlovski (Jira)
Stanislav Kozlovski created KAFKA-16140:
---

 Summary: 
zookeeper_migration_test#TestMigration#test_recooncile_kraft_to_zk system test 
fails concistently on 3.7
 Key: KAFKA-16140
 URL: https://issues.apache.org/jira/browse/KAFKA-16140
 Project: Kafka
  Issue Type: Test
Affects Versions: 3.7.0
Reporter: Stanislav Kozlovski


{code:java}
kafkatest.tests.core.

zookeeper_migration_test.TestMigration#test_reconcile_kraft_to_zk 

AssertionError('Did not see expected INFO log after migration')
Traceback (most recent call last):
  File 
"/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/venv/lib/python3.7/site-packages/ducktape/tests/runner_client.py",
 line 184, in _do_run
data = self.run_test()
  File 
"/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/venv/lib/python3.7/site-packages/ducktape/tests/runner_client.py",
 line 262, in run_test
return self.test_context.function(self.test)
  File 
"/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/tests/kafkatest/tests/core/zookeeper_migration_test.py",
 line 367, in test_reconcile_kraft_to_zk
assert saw_expected_log, "Did not see expected INFO log after migration"
AssertionError: Did not see expected INFO log after migration{code}
 



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


[jira] [Created] (KAFKA-16139) StreamsUpgradeTest fails consistently in 3.7.0

2024-01-15 Thread Stanislav Kozlovski (Jira)
Stanislav Kozlovski created KAFKA-16139:
---

 Summary: StreamsUpgradeTest fails consistently in 3.7.0
 Key: KAFKA-16139
 URL: https://issues.apache.org/jira/browse/KAFKA-16139
 Project: Kafka
  Issue Type: Test
Affects Versions: 3.7.0
Reporter: Stanislav Kozlovski


h1. 
kafkatest.tests.streams.streams_upgrade_test.StreamsUpgradeTest#test_rolling_upgrade_with_2_bouncesArguments:\{
 “from_version”: “3.5.1”, “to_version”: “3.7.0-SNAPSHOT”}
 
{{TimeoutError('Could not detect Kafka Streams version 3.7.0-SNAPSHOT on 
ubuntu@worker2')}}

 



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


[jira] [Created] (KAFKA-16138) QuotaTest system test fails consistently in 3.7

2024-01-15 Thread Stanislav Kozlovski (Jira)
Stanislav Kozlovski created KAFKA-16138:
---

 Summary: QuotaTest system test fails consistently in 3.7
 Key: KAFKA-16138
 URL: https://issues.apache.org/jira/browse/KAFKA-16138
 Project: Kafka
  Issue Type: Test
Affects Versions: 3.7.0
Reporter: Stanislav Kozlovski


as mentioned in 
[https://hackmd.io/@hOneAGCrSmKSpL8VF-1HWQ/HyRgRJmta#kafkatesttestsclientquota_testQuotaTesttest_quotaArguments-%E2%80%9Coverride_quota%E2%80%9D-true-%E2%80%9Cquota_type%E2%80%9D-%E2%80%9Cuser%E2%80%9D,]
 the test fails consistently:
{code:java}
ValueError('max() arg is an empty sequence')
Traceback (most recent call last):
  File 
"/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/venv/lib/python3.7/site-packages/ducktape/tests/runner_client.py",
 line 184, in _do_run
data = self.run_test()
  File 
"/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/venv/lib/python3.7/site-packages/ducktape/tests/runner_client.py",
 line 262, in run_test
return self.test_context.function(self.test)
  File 
"/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/venv/lib/python3.7/site-packages/ducktape/mark/_mark.py",
 line 433, in wrapper
return functools.partial(f, *args, **kwargs)(*w_args, **w_kwargs)
  File 
"/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/tests/kafkatest/tests/client/quota_test.py",
 line 169, in test_quota
success, msg = self.validate(self.kafka, producer, consumer)
  File 
"/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/tests/kafkatest/tests/client/quota_test.py",
 line 197, in validate
metric.value for k, metrics in producer.metrics(group='producer-metrics', 
name='outgoing-byte-rate', client_id=producer.client_id) for metric in metrics
ValueError: max() arg is an empty sequence {code}
 



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


[jira] [Created] (KAFKA-16137) ListClientMetricsResourcesResponse definition is missing field descriptions

2024-01-15 Thread Andrew Schofield (Jira)
Andrew Schofield created KAFKA-16137:


 Summary: ListClientMetricsResourcesResponse definition is missing 
field descriptions
 Key: KAFKA-16137
 URL: https://issues.apache.org/jira/browse/KAFKA-16137
 Project: Kafka
  Issue Type: Task
  Components: admin
Affects Versions: 3.7.0
Reporter: Andrew Schofield


This is purely improving the readability of the Kafka protocol documentation by 
adding missing description information for the fields of the 
`ListClientMetricsResources` response.



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


[REVIEW REQUEST] Move StorageTool to tools

2024-01-15 Thread Federico Valeri
Hi folks, this PR has been open for some time now, can anyone have a look?

https://github.com/apache/kafka/pull/14847

Thanks.


[jira] [Created] (KAFKA-16136) CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated() is very flaky

2024-01-15 Thread Stanislav Kozlovski (Jira)
Stanislav Kozlovski created KAFKA-16136:
---

 Summary: 
CoordinatorTest.testTaskRequestWithOldStartMsGetsUpdated() is very flaky
 Key: KAFKA-16136
 URL: https://issues.apache.org/jira/browse/KAFKA-16136
 Project: Kafka
  Issue Type: Test
Reporter: Stanislav Kozlovski


The test failed 3 builds in a row (with different JDK versions) in the 3.7 
release branch as part of verifying the release

Locally it passed

 



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


[jira] [Created] (KAFKA-16135) kafka.api.PlaintextConsumerTest.testPerPartitionLeadMetricsCleanUpWithSubscribe(String, String).quorum=kraft+kip848.groupProtocol=consumer is flaky

2024-01-15 Thread Stanislav Kozlovski (Jira)
Stanislav Kozlovski created KAFKA-16135:
---

 Summary: 
kafka.api.PlaintextConsumerTest.testPerPartitionLeadMetricsCleanUpWithSubscribe(String,
 String).quorum=kraft+kip848.groupProtocol=consumer is flaky
 Key: KAFKA-16135
 URL: https://issues.apache.org/jira/browse/KAFKA-16135
 Project: Kafka
  Issue Type: Test
Reporter: Stanislav Kozlovski


The test
kafka.api.PlaintextConsumerTest.testPerPartitionLeadMetricsCleanUpWithSubscribe(String,
 String).quorum=kraft+kip848.groupProtocol=consumer
is incredibly flaky - it failed 3 builds in a row for the 3.7 release 
candidate, but with different JDK versions. Locally it also fails often and 
requires a few retries to pass

 



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


[jira] [Created] (KAFKA-16134) kafka.api.PlaintextConsumerTest.testPerPartitionLagMetricsCleanUpWithSubscribe(String, String).quorum=kraft+kip848.groupProtocol=consumer is flaky

2024-01-15 Thread Stanislav Kozlovski (Jira)
Stanislav Kozlovski created KAFKA-16134:
---

 Summary: 
kafka.api.PlaintextConsumerTest.testPerPartitionLagMetricsCleanUpWithSubscribe(String,
 String).quorum=kraft+kip848.groupProtocol=consumer is flaky
 Key: KAFKA-16134
 URL: https://issues.apache.org/jira/browse/KAFKA-16134
 Project: Kafka
  Issue Type: Test
Reporter: Stanislav Kozlovski


The following test is very flaky. It failed 3 times consecutively in Jenkins 
runs for the 3.7 release candidate.
kafka.api.PlaintextConsumerTest.testPerPartitionLagMetricsCleanUpWithSubscribe(String,
 String).quorum=kraft+kip848.groupProtocol=consumer
 



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


Re: [REVIEW REQUEST] ConsumerGroupCommand move to tools

2024-01-15 Thread Николай Ижиков
Hello.

The PRs still need review.

Please, take a look - https://github.com/apache/kafka/pull/14856



> 12 дек. 2023 г., в 11:01, Николай Ижиков  написал(а):
> 
> Hello.
> 
> Please, take a time and help me with moving ConsumerGroupCommand move tools
> First patch in the series are waiting for review.
> 
> 
>> 4 дек. 2023 г., в 11:07, Николай Ижиков  написал(а):
>> 
>> Hello.
>> 
>> Dear Kafka committers.
>> As far as I know - we moving Kafka from scala to java.
>> 
>> I prepared a series of PRs to move `ConsumerGroupCommand` from scala to java.
>> From my experience, right now, Mickael Maison and Justine Olshan are 
>> overwhelmed with other activities and can review my patches from time to 
>> time only.
>> 
>> So, if any Kafka committer has any chance to review simple patches and keep 
>> pace of moving scala code to java, please, do it.
>> Help me to move these command to tools.
>> 
>> Looks like all code are prepared and quality review is all we need :).
>> 
>>> 30 нояб. 2023 г., в 13:28, Николай Ижиков  
>>> написал(а):
>>> 
>>> Hello.
>>> 
>>> I prepared a PR to move `ConsumerGroupCommand` to `tools` module.
>>> Full changes are pretty huge [1] so I split them into several PR’s.
>>> 
>>> Please, take a look at first PR in the series - 
>>> https://github.com/apache/kafka/pull/14856
>>> 
>>> PR adds `ConsumerGroupCommandOptions` class and other command case classes 
>>> to `tools` module.
>>> 
>>> [1] https://github.com/apache/kafka/pull/14471
>>> 
>> 
> 



Re: [PROPOSAL] Add commercial support page on website

2024-01-15 Thread Viktor Somogyi-Vass
Hi all,

I think that making and updating such a list implies tricky questions that
we have to deal with. Good examples were given by Matthias, that was my
first thought as well before reading his response.

On the other hand I think it would be a good alternative to create a list
of companies that are either regular or meaningful contributors to the
project, provide other help such as testing infra or sponsors of the
project in other ways. This may present those companies who provide support
too and I think it'd be some appreciation as well to display them.

What do you all think?

Best,
Viktor


On Mon, Jan 15, 2024 at 10:08 AM Francois Papon <
francois.pa...@openobject.fr> wrote:

> Hi Tison,
>
> Publishing a dedicated website for that can be a good idea, however if
> the link of the website could not be mention in the official Apache
> Kafka website I'm afraid that it will not be relevant.
>
> BTW, as I understand after all the feedback of the Apache Kafka PMC and
> community, my proposal is not a good idea for the project so I will
> close the PR.
>
> Thanks all for the feedback.
>
> regards,
>
> François
>
> On 14/01/2024 12:56, tison wrote:
> > FWIW - even if it's rejected by the Kafka PMC, you can maintain your
> > own page for such information and provide your personal comments on
> > them. If the object is to provide information and help users to make
> > decisions, it should help. Although you should do the SEO by yourself,
> > if the information is somehow neutral and valuable, you can ask the
> > @apachekafka Twitter (X) account to propagate it and provide a blog
> > for Kafka blogs.
> >
> > This is the common way how third-party "evangelist" producing content
> > and get it promoted.
> >
> > Best,
> > tison.
> >
> > Matthias J. Sax  于2024年1月13日周六 07:35写道:
> >> François,
> >>
> >> thanks for starting this initiative. Personally, I don't think it's
> >> necessarily harmful for the project to add such a new page, however, I
> >> share the same concerns others raised already.
> >>
> >> I understand your motivation that people had issues finding commercial
> >> support, but I am not sure we can address this issue that way. I am also
> >> "worried" (for the lack of a better word) that the page might become
> >> long an unwieldy. In the end, any freelancer/consultant offering Kafka
> >> services would be able to get on the page, so we might get hundreds of
> >> entries, what also makes it impossible for users to find what they are
> >> looking for. Also, the services of different companies might vary
> >> drastically; should users read all these descriptions? I can also
> >> imagine that some companies offer their services only in some
> >> countries/regions making it even harder for user to find what they are
> >> looking for?
> >>
> >> Overall, it sounds more like a search optimization problem, and thus it
> >> seems out-of-scope what we can solve. As I said, I am not strictly
> >> against it, but I just don't see much value either.
> >>
> >>
> >> -Matthias
> >>
> >> On 1/11/24 12:55 PM, Francois Papon wrote:
> >>> Hi Justine,
> >>>
> >>> You're right, Kafka is a part of my business (training, consulting,
> >>> architecture design, sla...) and most of the time, users/customers said
> >>> that it was hard for them to find a commercial support (in France for
> my
> >>> case) after searching on the Kafka website (Google didn't help them).
> >>>
> >>> As an ASF member and PMC of several ASF projects, I know that this kind
> >>> of page exist so this is why I made this proposal for the Kafka project
> >>> because I really think that it can help users.
> >>>
> >>> As you suggest, I can submit a PR to be added on the "powered by" page.
> >>>
> >>> Thanks,
> >>>
> >>> François
> >>>
> >>> On 11/01/2024 21:00, Justine Olshan wrote:
>  Hey François,
> 
>  My point was that the companies on that page use kafka as part of
> their
>  business. If you use Kafka as part of your business feel free to
> submit a
>  PR to be added.
> 
>  I second Chris's point that other projects are not enough to require
>  Kafka
>  having such a support page.
> 
>  Justine
> 
>  On Thu, Jan 11, 2024 at 11:57 AM Chris Egerton <
> fearthecel...@gmail.com>
>  wrote:
> 
> > Hi François,
> >
> > Is it an official policy of the ASF that projects provide a listing
> of
> > commercial support options for themselves? I understand that other
> > projects
> > have chosen to provide one, but this doesn't necessarily imply that
> all
> > projects should do the same, and I can't say I find this point very
> > convincing as a rebuttal to some of the good-faith concerns raised by
> > the
> > PMC and members of the community so far. However, if there's an
> official
> > ASF stance on this topic, then I acknowledge that Apache Kafka should
> > align
> > with it.
> >
> > Best,
> >
> > Chris
> >
> >
> > On Thu, 

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

2024-01-15 Thread Apache Jenkins Server
See 




[jira] [Created] (KAFKA-16133) Commits during reconciliation always time out

2024-01-15 Thread Lucas Brutschy (Jira)
Lucas Brutschy created KAFKA-16133:
--

 Summary: Commits during reconciliation always time out
 Key: KAFKA-16133
 URL: https://issues.apache.org/jira/browse/KAFKA-16133
 Project: Kafka
  Issue Type: Bug
  Components: clients, consumer
Affects Versions: 3.7.0
Reporter: Lucas Brutschy


This only affects the AsyncKafkaConsumer, which is in Preview in 3.7.

In MembershipManagerImpl there is a confusion between timeouts and deadlines. 
[https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java#L836C38-L836C38]

This causes all autocommits during reconciliation to immediately time out.



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


[jira] [Created] (KAFKA-16132) Upgrading from 3.6 to 3.7 in KRaft will have seconds of partitions unavailable

2024-01-15 Thread Luke Chen (Jira)
Luke Chen created KAFKA-16132:
-

 Summary: Upgrading from 3.6 to 3.7 in KRaft will have seconds of 
partitions unavailable
 Key: KAFKA-16132
 URL: https://issues.apache.org/jira/browse/KAFKA-16132
 Project: Kafka
  Issue Type: Bug
Affects Versions: 3.7.0
Reporter: Luke Chen


When upgrading from 3.6 to 3.7, we noticed that after upgrade the metadata 
version, all the partitions will be reset at one time, which causes a short 
period of time unavailable. This doesn't happen before. 


{code:java}
[2024-01-15 20:45:19,757] INFO [BrokerMetadataPublisher id=2] Updating 
metadata.version to 19 at offset OffsetAndEpoch(offset=229, epoch=2). 
(kafka.server.metadata.BrokerMetadataPublisher)
[2024-01-15 20:45:29,915] INFO [ReplicaFetcherManager on broker 2] Removed 
fetcher for partitions Set(t1-29, t1-25, t1-21, t1-17, t1-46, t1-13, t1-42, 
t1-9, t1-38, t1-5, t1-34, t1-1, t1-30, t1-26, t1-22, t1-18, t1-47, t1-14, 
t1-43, t1-10, t1-39, t1-6, t1-35, t1-2, t1-31, t1-27, t1-23, t1-19, t1-48, 
t1-15, t1-44, t1-11, t1-40, t1-7, t1-36, t1-3, t1-32, t1-28, t1-24, t1-20, 
t1-49, t1-16, t1-45, t1-12, t1-41, t1-8, t1-37, t1-4, t1-33, t1-0) 
(kafka.server.ReplicaFetcherManager)
{code}

Complete log:
https://gist.github.com/showuon/665aa3ce6afd59097a2662f8260ecc10

Steps:
1. start up a 3.6 kafka cluster in KRaft with 1 broker
2. create a topic
3. upgrade the binary to 3.7
4. use kafka-features.sh to upgrade to 3.7 metadata version
5. check the log (and metrics if interested)

Analysis:
In 3.7, we have JBOD support in KRaft, so the partitionRegistration added a new 
directory field. And it causes diff found while comparing delta. We might be 
able to identify this adding directory change doesn't need to reset the 
leader/follower state, and just update the metadata, to avoid causing 
unavailability. 





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


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

2024-01-15 Thread Christo Lolov
Heya!

Okay, your suggestion also makes sense to me!

I have updated the KIP.

Best,
Christo

On Mon, 15 Jan 2024 at 11:51, Luke Chen  wrote:

> Hi Christo,
>
> Thanks for the update.
> For "-4 or earliest-local" but tiered storage disabled, I agree it should
> work as requesting for as "-2 or earliest".
> For "-5 or latest-tiered" but tiered storage disabled, returning the
> earliest timestamp doesn't make sense to me.
> I'm thinking if we can return nothing, like what we did for this
> "[Note: No offset is returned, if the timestamp greater than recently
> committed record timestamp is given.]"
>
> WDYT?
>
> Thanks.
> Luke
>
> On Mon, Jan 15, 2024 at 6:46 PM Christo Lolov 
> wrote:
>
> > Heya Luke,
> >
> > Thank for the question! I have expanded in the KIP - in my opinion if -5
> > (latest-tiered) is requested when tiered storage is disabled Kafka should
> > return -2. My reasoning is that if there is no remote storage then we
> > should be returning an offset which is within the bounds of the log. Let
> me
> > know if you disagree!
> >
> > Best,
> > Christo
> >
> > On Fri, 12 Jan 2024 at 03:43, Luke Chen  wrote:
> >
> > > Hi Christo,
> > >
> > > Thanks for the KIP!
> > > One question:
> > >
> > > What will the offset return if tiered storage is disabled?
> > > For "-4 or earliest-local", it should be the same as "-2 or earliest",
> > > right?
> > > For "-5 or latest-tiered", it will be...0?
> > >
> > > I think the result should be written in the KIP (or script help text)
> > > explicitly.
> > >
> > > Thanks.
> > > Luke
> > >
> > > On Thu, Jan 11, 2024 at 6:54 PM Divij Vaidya 
> > > wrote:
> > >
> > > > Thank you for making the change Christo. It looks good to me.
> > > >
> > > > --
> > > > Divij Vaidya
> > > >
> > > >
> > > >
> > > > On Thu, Jan 11, 2024 at 11:19 AM Christo Lolov <
> christolo...@gmail.com
> > >
> > > > wrote:
> > > >
> > > > > Thank you Divij!
> > > > >
> > > > > I have updated the KIP to explicitly state that the broker will
> have
> > a
> > > > > different behaviour when a timestamp of -5 is requested as part of
> > > > > ListOffsets.
> > > > >
> > > > > Best,
> > > > > Christo
> > > > >
> > > > > On Tue, 2 Jan 2024 at 11:10, Divij Vaidya  >
> > > > wrote:
> > > > >
> > > > > > 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 <
> > > christolo...@gmail.com>
> > > > > > 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 <
> > > > satish.dugg...@gmail.com
> > > > > >
> > > > > > > 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!
> > > > > > > > >
> > > 

[VOTE] KIP-991: Allow DropHeaders SMT to drop headers by wildcard/regexp

2024-01-15 Thread Roman Schmitz
Hi all,

Thank you for your feedback on the suggested KIP so far.
As there seem to be no new updates / suggestions I'd like to start a vote
on the (SMT-) KIP 991.

Please have a look at KIP-991 .

Thanks,
Roman


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

2024-01-15 Thread Luke Chen
Hi Christo,

Thanks for the update.
For "-4 or earliest-local" but tiered storage disabled, I agree it should
work as requesting for as "-2 or earliest".
For "-5 or latest-tiered" but tiered storage disabled, returning the
earliest timestamp doesn't make sense to me.
I'm thinking if we can return nothing, like what we did for this
"[Note: No offset is returned, if the timestamp greater than recently
committed record timestamp is given.]"

WDYT?

Thanks.
Luke

On Mon, Jan 15, 2024 at 6:46 PM Christo Lolov 
wrote:

> Heya Luke,
>
> Thank for the question! I have expanded in the KIP - in my opinion if -5
> (latest-tiered) is requested when tiered storage is disabled Kafka should
> return -2. My reasoning is that if there is no remote storage then we
> should be returning an offset which is within the bounds of the log. Let me
> know if you disagree!
>
> Best,
> Christo
>
> On Fri, 12 Jan 2024 at 03:43, Luke Chen  wrote:
>
> > Hi Christo,
> >
> > Thanks for the KIP!
> > One question:
> >
> > What will the offset return if tiered storage is disabled?
> > For "-4 or earliest-local", it should be the same as "-2 or earliest",
> > right?
> > For "-5 or latest-tiered", it will be...0?
> >
> > I think the result should be written in the KIP (or script help text)
> > explicitly.
> >
> > Thanks.
> > Luke
> >
> > On Thu, Jan 11, 2024 at 6:54 PM Divij Vaidya 
> > wrote:
> >
> > > Thank you for making the change Christo. It looks good to me.
> > >
> > > --
> > > Divij Vaidya
> > >
> > >
> > >
> > > On Thu, Jan 11, 2024 at 11:19 AM Christo Lolov  >
> > > wrote:
> > >
> > > > Thank you Divij!
> > > >
> > > > I have updated the KIP to explicitly state that the broker will have
> a
> > > > different behaviour when a timestamp of -5 is requested as part of
> > > > ListOffsets.
> > > >
> > > > Best,
> > > > Christo
> > > >
> > > > On Tue, 2 Jan 2024 at 11:10, Divij Vaidya 
> > > wrote:
> > > >
> > > > > 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 <
> > christolo...@gmail.com>
> > > > > 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 <
> > > satish.dugg...@gmail.com
> > > > >
> > > > > > 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 <
> > > > > christolo...@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hello all!
> > > > > > > > >
> > > > > > > > > I would like to start a discussion for
> > > > > > 

[jira] [Resolved] (KAFKA-15740) KRaft support in DeleteOffsetsConsumerGroupCommandIntegrationTest

2024-01-15 Thread Mickael Maison (Jira)


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

Mickael Maison resolved KAFKA-15740.

Fix Version/s: 3.8.0
   Resolution: Fixed

> KRaft support in DeleteOffsetsConsumerGroupCommandIntegrationTest
> -
>
> Key: KAFKA-15740
> URL: https://issues.apache.org/jira/browse/KAFKA-15740
> Project: Kafka
>  Issue Type: Task
>  Components: core
>Reporter: Sameer Tejani
>Assignee: Zihao Lin
>Priority: Minor
>  Labels: kraft, kraft-test, newbie
> Fix For: 3.8.0
>
>
> The following tests in DeleteOffsetsConsumerGroupCommandIntegrationTest in 
> core/src/test/scala/unit/kafka/admin/DeleteOffsetsConsumerGroupCommandIntegrationTest.scala
>  need to be updated to support KRaft
> 49 : def testDeleteOffsetsNonExistingGroup(): Unit = {
> 59 : def testDeleteOffsetsOfStableConsumerGroupWithTopicPartition(): Unit = {
> 64 : def testDeleteOffsetsOfStableConsumerGroupWithTopicOnly(): Unit = {
> 69 : def testDeleteOffsetsOfStableConsumerGroupWithUnknownTopicPartition(): 
> Unit = {
> 74 : def testDeleteOffsetsOfStableConsumerGroupWithUnknownTopicOnly(): Unit = 
> {
> 79 : def testDeleteOffsetsOfEmptyConsumerGroupWithTopicPartition(): Unit = {
> 84 : def testDeleteOffsetsOfEmptyConsumerGroupWithTopicOnly(): Unit = {
> 89 : def testDeleteOffsetsOfEmptyConsumerGroupWithUnknownTopicPartition(): 
> Unit = {
> 94 : def testDeleteOffsetsOfEmptyConsumerGroupWithUnknownTopicOnly(): Unit = {
> Scanned 198 lines. Found 0 KRaft tests out of 9 tests



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


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

2024-01-15 Thread Christo Lolov
Heya Luke,

Thank for the question! I have expanded in the KIP - in my opinion if -5
(latest-tiered) is requested when tiered storage is disabled Kafka should
return -2. My reasoning is that if there is no remote storage then we
should be returning an offset which is within the bounds of the log. Let me
know if you disagree!

Best,
Christo

On Fri, 12 Jan 2024 at 03:43, Luke Chen  wrote:

> Hi Christo,
>
> Thanks for the KIP!
> One question:
>
> What will the offset return if tiered storage is disabled?
> For "-4 or earliest-local", it should be the same as "-2 or earliest",
> right?
> For "-5 or latest-tiered", it will be...0?
>
> I think the result should be written in the KIP (or script help text)
> explicitly.
>
> Thanks.
> Luke
>
> On Thu, Jan 11, 2024 at 6:54 PM Divij Vaidya 
> wrote:
>
> > Thank you for making the change Christo. It looks good to me.
> >
> > --
> > Divij Vaidya
> >
> >
> >
> > On Thu, Jan 11, 2024 at 11:19 AM Christo Lolov 
> > wrote:
> >
> > > Thank you Divij!
> > >
> > > I have updated the KIP to explicitly state that the broker will have a
> > > different behaviour when a timestamp of -5 is requested as part of
> > > ListOffsets.
> > >
> > > Best,
> > > Christo
> > >
> > > On Tue, 2 Jan 2024 at 11:10, Divij Vaidya 
> > wrote:
> > >
> > > > 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 <
> christolo...@gmail.com>
> > > > 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 <
> > satish.dugg...@gmail.com
> > > >
> > > > > 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 <
> > > > christolo...@gmail.com>
> > > > > > > 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: [VOTE] 3.7.0 RC2

2024-01-15 Thread Jakub Scholz
*> Hi Jakub,> > Thanks for trying the RC. I think what you found is a
blocker bug because it *
*> will generate huge amount of logspam. I guess we didn't find it in junit
tests *
*> since logspam doesn't fail the automated tests. But certainly it's not
suitable *
*> for production. Did you file a JIRA yet?*

Hi Colin,

I opened https://issues.apache.org/jira/browse/KAFKA-16131.

Thanks & Regards
Jakub

On Mon, Jan 15, 2024 at 8:57 AM Colin McCabe  wrote:

> Hi Stanislav,
>
> Thanks for making the first RC. The fact that it's titled RC2 is messing
> with my mind a bit. I hope this doesn't make people think that we're
> farther along than we are, heh.
>
> On Sun, Jan 14, 2024, at 13:54, Jakub Scholz wrote:
> > *> Nice catch! It does seem like we should have gated this behind the
> > metadata> version as KIP-858 implies. Is the cluster configured with
> > multiple log> dirs? What is the impact of the error messages?*
> >
> > I did not observe any obvious impact. I was able to send and receive
> > messages as normally. But to be honest, I have no idea what else
> > this might impact, so I did not try anything special.
> >
> > I think everyone upgrading an existing KRaft cluster will go through this
> > stage (running Kafka 3.7 with an older metadata version for at least a
> > while). So even if it is just a logged exception without any other
> impact I
> > wonder if it might scare users from upgrading. But I leave it to others
> to
> > decide if this is a blocker or not.
> >
>
> Hi Jakub,
>
> Thanks for trying the RC. I think what you found is a blocker bug because
> it will generate huge amount of logspam. I guess we didn't find it in junit
> tests since logspam doesn't fail the automated tests. But certainly it's
> not suitable for production. Did you file a JIRA yet?
>
> > On Sun, Jan 14, 2024 at 10:17 PM Stanislav Kozlovski
> >  wrote:
> >
> >> Hey Luke,
> >>
> >> This is an interesting problem. Given the fact that the KIP for having a
> >> 3.8 release passed, I think it weights the scale towards not calling
> this a
> >> blocker and expecting it to be solved in 3.7.1.
> >>
> >> It is unfortunate that it would not seem safe to migrate to KRaft in
> 3.7.0
> >> (given the inability to rollback safely), but if that's true - the same
> >> case would apply for 3.6.0. So in any case users w\ould be expected to
> use a
> >> patch release for this.
>
> Hi Luke,
>
> Thanks for testing rollback. I think this is a case where the
> documentation is wrong. The intention was to for the steps to basically be:
>
> 1. roll all the brokers into zk mode, but with migration enabled
> 2. take down the kraft quorum
> 3. rmr /controller, allowing a hybrid broker to take over.
> 4. roll all the brokers into zk mode without migration enabled (if desired)
>
> With these steps, there isn't really unavailability since a ZK controller
> can be elected quickly after the kraft quorum is gone.
>
> >> Further, since we will have a 3.8 release - it is
> >> likely we will ultimately recommend users upgrade from that version
> given
> >> its aim is to have strategic KRaft feature parity with ZK.
> >> That being said, I am not 100% on this. Let me know whether you think
> this
> >> should block the release, Luke. I am also tagging Colin and David to
> weigh
> >> in with their opinions, as they worked on the migration logic.
>
> The rollback docs are new in 3.7 so the fact that they're wrong is a clear
> blocker, I think. But easy to fix, I believe. I will create a PR.
>
> best,
> Colin
>
> >>
> >> Hey Kirk and Chris,
> >>
> >> Unless I'm missing something - KAFKALESS-16029 is simply a bad log due
> to
> >> improper closing. And the PR description implies this has been present
> >> since 3.5. While annoying, I don't see a strong reason for this to block
> >> the release.
> >>
> >> Hey Jakub,
> >>
> >> Nice catch! It does seem like we should have gated this behind the
> metadata
> >> version as KIP-858 implies. Is the cluster configured with multiple log
> >> dirs? What is the impact of the error messages?
> >>
> >> Tagging Igor (the author of the KIP) to weigh in.
> >>
> >> Best,
> >> Stanislav
> >>
> >> On Sat, Jan 13, 2024 at 7:22 PM Jakub Scholz  wrote:
> >>
> >> > Hi,
> >> >
> >> > I was trying the RC2 and run into the following issue ... when I run
> >> > 3.7.0-RC2 KRaft cluster with metadata version set to 3.6-IV2 metadata
> >> > version, I seem to be getting repeated errors like this in the
> controller
> >> > logs:
> >> >
> >> > 2024-01-13 16:58:01,197 INFO [QuorumController id=0]
> >> assignReplicasToDirs:
> >> > event failed with UnsupportedVersionException in 15 microseconds.
> >> > (org.apache.kafka.controller.QuorumController)
> >> > [quorum-controller-0-event-handler]
> >> > 2024-01-13 16:58:01,197 ERROR [ControllerApis nodeId=0] Unexpected
> error
> >> > handling request RequestHeader(apiKey=ASSIGN_REPLICAS_TO_DIRS,
> >> > apiVersion=0, clientId=1000, correlationId=14, headerVersion=2) --
> >> > 

[jira] [Created] (KAFKA-16131) Repeated UnsupportedVersionException logged when running Kafka 3.7.0-RC2 KRaft cluster with metadata version 3.6

2024-01-15 Thread Jakub Scholz (Jira)
Jakub Scholz created KAFKA-16131:


 Summary: Repeated UnsupportedVersionException logged when running 
Kafka 3.7.0-RC2 KRaft cluster with metadata version 3.6
 Key: KAFKA-16131
 URL: https://issues.apache.org/jira/browse/KAFKA-16131
 Project: Kafka
  Issue Type: Bug
Reporter: Jakub Scholz
 Fix For: 3.7.0


When running Kafka 3.7.0-RC2 as a KRaft cluster with metadata version set to 
3.6-IV2 metadata version, it throws repeated errors like this in the controller 
logs:
{quote}2024-01-13 16:58:01,197 INFO [QuorumController id=0] 
assignReplicasToDirs: event failed with UnsupportedVersionException in 15 
microseconds. (org.apache.kafka.controller.QuorumController) 
[quorum-controller-0-event-handler]
2024-01-13 16:58:01,197 ERROR [ControllerApis nodeId=0] Unexpected error 
handling request RequestHeader(apiKey=ASSIGN_REPLICAS_TO_DIRS, apiVersion=0, 
clientId=1000, correlationId=14, headerVersion=2) – 
AssignReplicasToDirsRequestData(brokerId=1000, brokerEpoch=5, 
directories=[DirectoryData(id=w_uxN7pwQ6eXSMrOKceYIQ, 
topics=[TopicData(topicId=bvAKLSwmR7iJoKv2yZgygQ, 
partitions=[PartitionData(partitionIndex=2), PartitionData(partitionIndex=1)]), 
TopicData(topicId=uNe7f5VrQgO0zST6yH1jDQ, 
partitions=[PartitionData(partitionIndex=0)])])]) with context 
RequestContext(header=RequestHeader(apiKey=ASSIGN_REPLICAS_TO_DIRS, 
apiVersion=0, clientId=1000, correlationId=14, headerVersion=2), 
connectionId='172.16.14.219:9090-172.16.14.217:53590-7', 
clientAddress=/[172.16.14.217|http://172.16.14.217/], 
principal=User:CN=my-cluster-kafka,O=io.strimzi, 
listenerName=ListenerName(CONTROLPLANE-9090), securityProtocol=SSL, 
clientInformation=ClientInformation(softwareName=apache-kafka-java, 
softwareVersion=3.7.0), fromPrivilegedListener=false, 
principalSerde=Optional[org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder@71004ad2])
 (kafka.server.ControllerApis) [quorum-controller-0-event-handler]
java.util.concurrent.CompletionException: 
org.apache.kafka.common.errors.UnsupportedVersionException: Directory 
assignment is not supported yet.
at 
java.base/java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:332)
 at 
java.base/java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:347)
 at 
java.base/java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:636)
 at 
java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
 at 
java.base/java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2162)
 at 
org.apache.kafka.controller.QuorumController$ControllerWriteEvent.complete(QuorumController.java:880)
 at 
org.apache.kafka.controller.QuorumController$ControllerWriteEvent.handleException(QuorumController.java:871)
 at 
org.apache.kafka.queue.KafkaEventQueue$EventContext.completeWithException(KafkaEventQueue.java:148)
 at 
org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:137)
 at 
org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:210)
 at 
org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:181)
 at java.base/java.lang.Thread.run(Thread.java:840)
Caused by: org.apache.kafka.common.errors.UnsupportedVersionException: 
Directory assignment is not supported yet.
{quote}
 
With the metadata version set to 3.6-IV2, it makes sense that the request is 
not supported. But the request should in such case not be sent at all.



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


Re: [PR] Add get support page [kafka-site]

2024-01-15 Thread via GitHub


fpapon commented on PR #577:
URL: https://github.com/apache/kafka-site/pull/577#issuecomment-1891670491

   > @fpapon I guess you mean Kafka PMC :) Not Karaf :)
   
   Ah yes, sorry, split brain :)
   I fixed.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] Add get support page [kafka-site]

2024-01-15 Thread via GitHub


jbonofre commented on PR #577:
URL: https://github.com/apache/kafka-site/pull/577#issuecomment-1891649260

   @fpapon I guess you mean Kafka PMC :) Not Karaf :) 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] Add get support page [kafka-site]

2024-01-15 Thread via GitHub


fpapon commented on PR #577:
URL: https://github.com/apache/kafka-site/pull/577#issuecomment-1891642675

   As the Apache Karaf PMC rejeted this proposal on the mailing list, I'm 
closing this PR.
   Link to the thread:
   https://lists.apache.org/thread/sld85ly7fvvvlb5bh2856qqcpnckg2on


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] Add get support page [kafka-site]

2024-01-15 Thread via GitHub


fpapon closed pull request #577: Add get support page
URL: https://github.com/apache/kafka-site/pull/577


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[jira] [Created] (KAFKA-16130) Test migration rollback

2024-01-15 Thread Mickael Maison (Jira)
Mickael Maison created KAFKA-16130:
--

 Summary: Test migration rollback
 Key: KAFKA-16130
 URL: https://issues.apache.org/jira/browse/KAFKA-16130
 Project: Kafka
  Issue Type: Sub-task
Reporter: Mickael Maison
 Fix For: 3.8.0






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


Re: [PROPOSAL] Add commercial support page on website

2024-01-15 Thread Francois Papon

Hi Tison,

Publishing a dedicated website for that can be a good idea, however if 
the link of the website could not be mention in the official Apache 
Kafka website I'm afraid that it will not be relevant.


BTW, as I understand after all the feedback of the Apache Kafka PMC and 
community, my proposal is not a good idea for the project so I will 
close the PR.


Thanks all for the feedback.

regards,

François

On 14/01/2024 12:56, tison wrote:

FWIW - even if it's rejected by the Kafka PMC, you can maintain your
own page for such information and provide your personal comments on
them. If the object is to provide information and help users to make
decisions, it should help. Although you should do the SEO by yourself,
if the information is somehow neutral and valuable, you can ask the
@apachekafka Twitter (X) account to propagate it and provide a blog
for Kafka blogs.

This is the common way how third-party "evangelist" producing content
and get it promoted.

Best,
tison.

Matthias J. Sax  于2024年1月13日周六 07:35写道:

François,

thanks for starting this initiative. Personally, I don't think it's
necessarily harmful for the project to add such a new page, however, I
share the same concerns others raised already.

I understand your motivation that people had issues finding commercial
support, but I am not sure we can address this issue that way. I am also
"worried" (for the lack of a better word) that the page might become
long an unwieldy. In the end, any freelancer/consultant offering Kafka
services would be able to get on the page, so we might get hundreds of
entries, what also makes it impossible for users to find what they are
looking for. Also, the services of different companies might vary
drastically; should users read all these descriptions? I can also
imagine that some companies offer their services only in some
countries/regions making it even harder for user to find what they are
looking for?

Overall, it sounds more like a search optimization problem, and thus it
seems out-of-scope what we can solve. As I said, I am not strictly
against it, but I just don't see much value either.


-Matthias

On 1/11/24 12:55 PM, Francois Papon wrote:

Hi Justine,

You're right, Kafka is a part of my business (training, consulting,
architecture design, sla...) and most of the time, users/customers said
that it was hard for them to find a commercial support (in France for my
case) after searching on the Kafka website (Google didn't help them).

As an ASF member and PMC of several ASF projects, I know that this kind
of page exist so this is why I made this proposal for the Kafka project
because I really think that it can help users.

As you suggest, I can submit a PR to be added on the "powered by" page.

Thanks,

François

On 11/01/2024 21:00, Justine Olshan wrote:

Hey François,

My point was that the companies on that page use kafka as part of their
business. If you use Kafka as part of your business feel free to submit a
PR to be added.

I second Chris's point that other projects are not enough to require
Kafka
having such a support page.

Justine

On Thu, Jan 11, 2024 at 11:57 AM Chris Egerton 
wrote:


Hi François,

Is it an official policy of the ASF that projects provide a listing of
commercial support options for themselves? I understand that other
projects
have chosen to provide one, but this doesn't necessarily imply that all
projects should do the same, and I can't say I find this point very
convincing as a rebuttal to some of the good-faith concerns raised by
the
PMC and members of the community so far. However, if there's an official
ASF stance on this topic, then I acknowledge that Apache Kafka should
align
with it.

Best,

Chris


On Thu, Jan 11, 2024, 14:50 fpapon  wrote:


Hi Justine,

I'm not sure to see the difference between "happy users" and vendors
that advertise their products in some of the company list in the
"powered by" page.

Btw, my initial purpose of my proposal was to help user to find support
for production stuff rather than searching in google.

I don't think this is a bad thing because this is something that
already
exist in many ASF projects like:

https://hop.apache.org/community/commercial/
https://struts.apache.org/commercial-support.html
https://directory.apache.org/commercial-support.html
https://tomee.apache.org/commercial-support.html
https://plc4x.apache.org/users/commercial-support.html
https://camel.apache.org/community/support/
https://openmeetings.apache.org/commercial-support.html
https://guacamole.apache.org/support/



https://cwiki.apache.org/confluence/display/HADOOP2/Distributions+and+Commercial+Support
https://activemq.apache.org/supporthttps://karaf.apache.org/community.html

https://netbeans.apache.org/front/main/help/commercial-support/
https://royale.apache.org/royale-commercial-support/

https://karaf.apache.org/community.html

As I understand for now, the channel for users to find production
support is:

- The mailing list (u...@kafka.apache.org / 

Re: [PROPOSAL] Add commercial support page on website

2024-01-15 Thread Francois Papon

Hi Matthias,

Thank you for your feedback, it make sense. My proposal was to help user 
finding support but as you said, may be this is not the right way to 
address it.


regards,

François

On 13/01/2024 00:34, Matthias J. Sax wrote:

François,

thanks for starting this initiative. Personally, I don't think it's 
necessarily harmful for the project to add such a new page, however, I 
share the same concerns others raised already.


I understand your motivation that people had issues finding commercial 
support, but I am not sure we can address this issue that way. I am 
also "worried" (for the lack of a better word) that the page might 
become long an unwieldy. In the end, any freelancer/consultant 
offering Kafka services would be able to get on the page, so we might 
get hundreds of entries, what also makes it impossible for users to 
find what they are looking for. Also, the services of different 
companies might vary drastically; should users read all these 
descriptions? I can also imagine that some companies offer their 
services only in some countries/regions making it even harder for user 
to find what they are looking for?


Overall, it sounds more like a search optimization problem, and thus 
it seems out-of-scope what we can solve. As I said, I am not strictly 
against it, but I just don't see much value either.



-Matthias

On 1/11/24 12:55 PM, Francois Papon wrote:

Hi Justine,

You're right, Kafka is a part of my business (training, consulting, 
architecture design, sla...) and most of the time, users/customers 
said that it was hard for them to find a commercial support (in 
France for my case) after searching on the Kafka website (Google 
didn't help them).


As an ASF member and PMC of several ASF projects, I know that this 
kind of page exist so this is why I made this proposal for the Kafka 
project because I really think that it can help users.


As you suggest, I can submit a PR to be added on the "powered by" page.

Thanks,

François

On 11/01/2024 21:00, Justine Olshan wrote:

Hey François,

My point was that the companies on that page use kafka as part of their
business. If you use Kafka as part of your business feel free to 
submit a

PR to be added.

I second Chris's point that other projects are not enough to require 
Kafka

having such a support page.

Justine

On Thu, Jan 11, 2024 at 11:57 AM Chris Egerton 


wrote:


Hi François,

Is it an official policy of the ASF that projects provide a listing of
commercial support options for themselves? I understand that other 
projects
have chosen to provide one, but this doesn't necessarily imply that 
all

projects should do the same, and I can't say I find this point very
convincing as a rebuttal to some of the good-faith concerns raised 
by the
PMC and members of the community so far. However, if there's an 
official
ASF stance on this topic, then I acknowledge that Apache Kafka 
should align

with it.

Best,

Chris


On Thu, Jan 11, 2024, 14:50 fpapon  wrote:


Hi Justine,

I'm not sure to see the difference between "happy users" and vendors
that advertise their products in some of the company list in the
"powered by" page.

Btw, my initial purpose of my proposal was to help user to find 
support

for production stuff rather than searching in google.

I don't think this is a bad thing because this is something that 
already

exist in many ASF projects like:

https://hop.apache.org/community/commercial/
https://struts.apache.org/commercial-support.html
https://directory.apache.org/commercial-support.html
https://tomee.apache.org/commercial-support.html
https://plc4x.apache.org/users/commercial-support.html
https://camel.apache.org/community/support/
https://openmeetings.apache.org/commercial-support.html
https://guacamole.apache.org/support/


https://cwiki.apache.org/confluence/display/HADOOP2/Distributions+and+Commercial+Support 

https://activemq.apache.org/supporthttps://karaf.apache.org/community.html 


https://netbeans.apache.org/front/main/help/commercial-support/
https://royale.apache.org/royale-commercial-support/

https://karaf.apache.org/community.html

As I understand for now, the channel for users to find production
support is:

- The mailing list (u...@kafka.apache.org / dev@kafka.apache.org)

- The official #kafka  ASF Slack channel (may be we can add it on the
website because I didn't find it in the website =>
https://kafka.apache.org/contact)

- Search in google for commercial support only

I can update my PR to mention only the 3 points above for the "get
support" page if people think that having a support page make sense.

regards,

François

On 11/01/2024 19:34, Justine Olshan wrote:
I think there is a difference between the "Powered by" page and a 
page

for

vendors to advertise their products and services.

The idea is that the companies on that page are "powered by" Kafka.

They

serve as examples of happy users of Kafka.
I don't think it is meant only as a place just for those 
companies to

advertise.

Re: Permission to contribute

2024-01-15 Thread Josep Prat
Hi Rowland,

Thanks for your interest in Apache Kafka! Your accounts are all set. Let me
know if you have any questions.

Best,

On Sun, Jan 14, 2024 at 2:17 AM Rowland Smith  wrote:

> I would like permission to contribute to Kafka. I have created Wiki and
> Jira ID's 'rowls'.
>
> I will be working with a KIP for XA support.
>
> --
> *Rowland E. Smith*
> P: (862) 260-4163
> M: (201) 396-3842
>


-- 
[image: Aiven] 

*Josep Prat*
Open Source Engineering Director, *Aiven*
josep.p...@aiven.io   |   +491715557497
aiven.io    |   
     
*Aiven Deutschland GmbH*
Alexanderufer 3-7, 10117 Berlin
Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen
Amtsgericht Charlottenburg, HRB 209739 B


Re: [VOTE] 3.7.0 RC2

2024-01-15 Thread Luke Chen
Hi Paolo, Colin,

Let's discuss detail about this issue in the PR:
https://github.com/apache/kafka/pull/15193 .

On Mon, Jan 15, 2024 at 4:21 PM Paolo Patierno 
wrote:

> Hi Colin,
> I was the one raising the issue about rollback and I also already tried
> what you mentioned but with no success.
> During the first rolling, I left
> the zookeeper.metadata.migration.enable=true but
> removed controller.quorum.voters and controller.listener.names.
> This is what I get from the brokers on restarting:
>
> 2024-01-15 09:19:14,172] ERROR Exiting Kafka due to fatal exception
> (kafka.Kafka$)
> org.apache.kafka.common.config.ConfigException: If using
> zookeeper.metadata.migration.enable, controller.quorum.voters must contain
> a parseable set of voters.
> at
>
> kafka.server.KafkaConfig.validateNonEmptyQuorumVotersForMigration$1(KafkaConfig.scala:2286)
> at kafka.server.KafkaConfig.validateValues(KafkaConfig.scala:2371)
> at kafka.server.KafkaConfig.(KafkaConfig.scala:2233)
> at kafka.server.KafkaConfig.(KafkaConfig.scala:1604)
> at kafka.server.KafkaConfig$.fromProps(KafkaConfig.scala:1527)
> at kafka.Kafka$.buildServer(Kafka.scala:72)
> at kafka.Kafka$.main(Kafka.scala:91)
> at kafka.Kafka.main(Kafka.scala)
>
> Did you try it?
> Am I missing anything in your procedure?
>
> Thanks,
> Paolo
>
> On Mon, 15 Jan 2024 at 09:13, Colin McCabe  wrote:
>
> > Docs fix discussed in the thread is here:
> > https://github.com/apache/kafka/pull/15193
> >
> > best,
> > Colin
> >
> >
> > On Sun, Jan 14, 2024, at 23:56, Colin McCabe wrote:
> > > Hi Stanislav,
> > >
> > > Thanks for making the first RC. The fact that it's titled RC2 is
> > > messing with my mind a bit. I hope this doesn't make people think that
> > > we're farther along than we are, heh.
> > >
> > > On Sun, Jan 14, 2024, at 13:54, Jakub Scholz wrote:
> > >> *> Nice catch! It does seem like we should have gated this behind the
> > >> metadata> version as KIP-858 implies. Is the cluster configured with
> > >> multiple log> dirs? What is the impact of the error messages?*
> > >>
> > >> I did not observe any obvious impact. I was able to send and receive
> > >> messages as normally. But to be honest, I have no idea what else
> > >> this might impact, so I did not try anything special.
> > >>
> > >> I think everyone upgrading an existing KRaft cluster will go through
> > this
> > >> stage (running Kafka 3.7 with an older metadata version for at least a
> > >> while). So even if it is just a logged exception without any other
> > impact I
> > >> wonder if it might scare users from upgrading. But I leave it to
> others
> > to
> > >> decide if this is a blocker or not.
> > >>
> > >
> > > Hi Jakub,
> > >
> > > Thanks for trying the RC. I think what you found is a blocker bug
> > > because it will generate huge amount of logspam. I guess we didn't find
> > > it in junit tests since logspam doesn't fail the automated tests. But
> > > certainly it's not suitable for production. Did you file a JIRA yet?
> > >
> > >> On Sun, Jan 14, 2024 at 10:17 PM Stanislav Kozlovski
> > >>  wrote:
> > >>
> > >>> Hey Luke,
> > >>>
> > >>> This is an interesting problem. Given the fact that the KIP for
> having
> > a
> > >>> 3.8 release passed, I think it weights the scale towards not calling
> > this a
> > >>> blocker and expecting it to be solved in 3.7.1.
> > >>>
> > >>> It is unfortunate that it would not seem safe to migrate to KRaft in
> > 3.7.0
> > >>> (given the inability to rollback safely), but if that's true - the
> same
> > >>> case would apply for 3.6.0. So in any case users w\ould be expected
> to
> > use a
> > >>> patch release for this.
> > >
> > > Hi Luke,
> > >
> > > Thanks for testing rollback. I think this is a case where the
> > > documentation is wrong. The intention was to for the steps to basically
> > > be:
> > >
> > > 1. roll all the brokers into zk mode, but with migration enabled
> > > 2. take down the kraft quorum
> > > 3. rmr /controller, allowing a hybrid broker to take over.
> > > 4. roll all the brokers into zk mode without migration enabled (if
> > desired)
> > >
> > > With these steps, there isn't really unavailability since a ZK
> > > controller can be elected quickly after the kraft quorum is gone.
> > >
> > >>> Further, since we will have a 3.8 release - it is
> > >>> likely we will ultimately recommend users upgrade from that version
> > given
> > >>> its aim is to have strategic KRaft feature parity with ZK.
> > >>> That being said, I am not 100% on this. Let me know whether you think
> > this
> > >>> should block the release, Luke. I am also tagging Colin and David to
> > weigh
> > >>> in with their opinions, as they worked on the migration logic.
> > >
> > > The rollback docs are new in 3.7 so the fact that they're wrong is a
> > > clear blocker, I think. But easy to fix, I believe. I will create a PR.
> > >
> > > best,
> > > Colin
> > >
> > >>>
> > >>> Hey Kirk and Chris,
> > >>>
> > >>> 

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

2024-01-15 Thread Apache Jenkins Server
See 




Re: [VOTE] 3.7.0 RC2

2024-01-15 Thread Paolo Patierno
Hi Colin,
I was the one raising the issue about rollback and I also already tried
what you mentioned but with no success.
During the first rolling, I left
the zookeeper.metadata.migration.enable=true but
removed controller.quorum.voters and controller.listener.names.
This is what I get from the brokers on restarting:

2024-01-15 09:19:14,172] ERROR Exiting Kafka due to fatal exception
(kafka.Kafka$)
org.apache.kafka.common.config.ConfigException: If using
zookeeper.metadata.migration.enable, controller.quorum.voters must contain
a parseable set of voters.
at
kafka.server.KafkaConfig.validateNonEmptyQuorumVotersForMigration$1(KafkaConfig.scala:2286)
at kafka.server.KafkaConfig.validateValues(KafkaConfig.scala:2371)
at kafka.server.KafkaConfig.(KafkaConfig.scala:2233)
at kafka.server.KafkaConfig.(KafkaConfig.scala:1604)
at kafka.server.KafkaConfig$.fromProps(KafkaConfig.scala:1527)
at kafka.Kafka$.buildServer(Kafka.scala:72)
at kafka.Kafka$.main(Kafka.scala:91)
at kafka.Kafka.main(Kafka.scala)

Did you try it?
Am I missing anything in your procedure?

Thanks,
Paolo

On Mon, 15 Jan 2024 at 09:13, Colin McCabe  wrote:

> Docs fix discussed in the thread is here:
> https://github.com/apache/kafka/pull/15193
>
> best,
> Colin
>
>
> On Sun, Jan 14, 2024, at 23:56, Colin McCabe wrote:
> > Hi Stanislav,
> >
> > Thanks for making the first RC. The fact that it's titled RC2 is
> > messing with my mind a bit. I hope this doesn't make people think that
> > we're farther along than we are, heh.
> >
> > On Sun, Jan 14, 2024, at 13:54, Jakub Scholz wrote:
> >> *> Nice catch! It does seem like we should have gated this behind the
> >> metadata> version as KIP-858 implies. Is the cluster configured with
> >> multiple log> dirs? What is the impact of the error messages?*
> >>
> >> I did not observe any obvious impact. I was able to send and receive
> >> messages as normally. But to be honest, I have no idea what else
> >> this might impact, so I did not try anything special.
> >>
> >> I think everyone upgrading an existing KRaft cluster will go through
> this
> >> stage (running Kafka 3.7 with an older metadata version for at least a
> >> while). So even if it is just a logged exception without any other
> impact I
> >> wonder if it might scare users from upgrading. But I leave it to others
> to
> >> decide if this is a blocker or not.
> >>
> >
> > Hi Jakub,
> >
> > Thanks for trying the RC. I think what you found is a blocker bug
> > because it will generate huge amount of logspam. I guess we didn't find
> > it in junit tests since logspam doesn't fail the automated tests. But
> > certainly it's not suitable for production. Did you file a JIRA yet?
> >
> >> On Sun, Jan 14, 2024 at 10:17 PM Stanislav Kozlovski
> >>  wrote:
> >>
> >>> Hey Luke,
> >>>
> >>> This is an interesting problem. Given the fact that the KIP for having
> a
> >>> 3.8 release passed, I think it weights the scale towards not calling
> this a
> >>> blocker and expecting it to be solved in 3.7.1.
> >>>
> >>> It is unfortunate that it would not seem safe to migrate to KRaft in
> 3.7.0
> >>> (given the inability to rollback safely), but if that's true - the same
> >>> case would apply for 3.6.0. So in any case users w\ould be expected to
> use a
> >>> patch release for this.
> >
> > Hi Luke,
> >
> > Thanks for testing rollback. I think this is a case where the
> > documentation is wrong. The intention was to for the steps to basically
> > be:
> >
> > 1. roll all the brokers into zk mode, but with migration enabled
> > 2. take down the kraft quorum
> > 3. rmr /controller, allowing a hybrid broker to take over.
> > 4. roll all the brokers into zk mode without migration enabled (if
> desired)
> >
> > With these steps, there isn't really unavailability since a ZK
> > controller can be elected quickly after the kraft quorum is gone.
> >
> >>> Further, since we will have a 3.8 release - it is
> >>> likely we will ultimately recommend users upgrade from that version
> given
> >>> its aim is to have strategic KRaft feature parity with ZK.
> >>> That being said, I am not 100% on this. Let me know whether you think
> this
> >>> should block the release, Luke. I am also tagging Colin and David to
> weigh
> >>> in with their opinions, as they worked on the migration logic.
> >
> > The rollback docs are new in 3.7 so the fact that they're wrong is a
> > clear blocker, I think. But easy to fix, I believe. I will create a PR.
> >
> > best,
> > Colin
> >
> >>>
> >>> Hey Kirk and Chris,
> >>>
> >>> Unless I'm missing something - KAFKALESS-16029 is simply a bad log due
> to
> >>> improper closing. And the PR description implies this has been present
> >>> since 3.5. While annoying, I don't see a strong reason for this to
> block
> >>> the release.
> >>>
> >>> Hey Jakub,
> >>>
> >>> Nice catch! It does seem like we should have gated this behind the
> metadata
> >>> version as KIP-858 implies. Is the cluster 

Re: [VOTE] 3.7.0 RC2

2024-01-15 Thread Colin McCabe
Docs fix discussed in the thread is here: 
https://github.com/apache/kafka/pull/15193

best,
Colin


On Sun, Jan 14, 2024, at 23:56, Colin McCabe wrote:
> Hi Stanislav,
>
> Thanks for making the first RC. The fact that it's titled RC2 is 
> messing with my mind a bit. I hope this doesn't make people think that 
> we're farther along than we are, heh.
>
> On Sun, Jan 14, 2024, at 13:54, Jakub Scholz wrote:
>> *> Nice catch! It does seem like we should have gated this behind the
>> metadata> version as KIP-858 implies. Is the cluster configured with
>> multiple log> dirs? What is the impact of the error messages?*
>>
>> I did not observe any obvious impact. I was able to send and receive
>> messages as normally. But to be honest, I have no idea what else
>> this might impact, so I did not try anything special.
>>
>> I think everyone upgrading an existing KRaft cluster will go through this
>> stage (running Kafka 3.7 with an older metadata version for at least a
>> while). So even if it is just a logged exception without any other impact I
>> wonder if it might scare users from upgrading. But I leave it to others to
>> decide if this is a blocker or not.
>>
>
> Hi Jakub,
>
> Thanks for trying the RC. I think what you found is a blocker bug 
> because it will generate huge amount of logspam. I guess we didn't find 
> it in junit tests since logspam doesn't fail the automated tests. But 
> certainly it's not suitable for production. Did you file a JIRA yet?
>
>> On Sun, Jan 14, 2024 at 10:17 PM Stanislav Kozlovski
>>  wrote:
>>
>>> Hey Luke,
>>>
>>> This is an interesting problem. Given the fact that the KIP for having a
>>> 3.8 release passed, I think it weights the scale towards not calling this a
>>> blocker and expecting it to be solved in 3.7.1.
>>>
>>> It is unfortunate that it would not seem safe to migrate to KRaft in 3.7.0
>>> (given the inability to rollback safely), but if that's true - the same
>>> case would apply for 3.6.0. So in any case users w\ould be expected to use a
>>> patch release for this.
>
> Hi Luke,
>
> Thanks for testing rollback. I think this is a case where the 
> documentation is wrong. The intention was to for the steps to basically 
> be:
>
> 1. roll all the brokers into zk mode, but with migration enabled
> 2. take down the kraft quorum
> 3. rmr /controller, allowing a hybrid broker to take over.
> 4. roll all the brokers into zk mode without migration enabled (if desired)
>
> With these steps, there isn't really unavailability since a ZK 
> controller can be elected quickly after the kraft quorum is gone.
>
>>> Further, since we will have a 3.8 release - it is
>>> likely we will ultimately recommend users upgrade from that version given
>>> its aim is to have strategic KRaft feature parity with ZK.
>>> That being said, I am not 100% on this. Let me know whether you think this
>>> should block the release, Luke. I am also tagging Colin and David to weigh
>>> in with their opinions, as they worked on the migration logic.
>
> The rollback docs are new in 3.7 so the fact that they're wrong is a 
> clear blocker, I think. But easy to fix, I believe. I will create a PR.
>
> best,
> Colin
>
>>>
>>> Hey Kirk and Chris,
>>>
>>> Unless I'm missing something - KAFKALESS-16029 is simply a bad log due to
>>> improper closing. And the PR description implies this has been present
>>> since 3.5. While annoying, I don't see a strong reason for this to block
>>> the release.
>>>
>>> Hey Jakub,
>>>
>>> Nice catch! It does seem like we should have gated this behind the metadata
>>> version as KIP-858 implies. Is the cluster configured with multiple log
>>> dirs? What is the impact of the error messages?
>>>
>>> Tagging Igor (the author of the KIP) to weigh in.
>>>
>>> Best,
>>> Stanislav
>>>
>>> On Sat, Jan 13, 2024 at 7:22 PM Jakub Scholz  wrote:
>>>
>>> > Hi,
>>> >
>>> > I was trying the RC2 and run into the following issue ... when I run
>>> > 3.7.0-RC2 KRaft cluster with metadata version set to 3.6-IV2 metadata
>>> > version, I seem to be getting repeated errors like this in the controller
>>> > logs:
>>> >
>>> > 2024-01-13 16:58:01,197 INFO [QuorumController id=0]
>>> assignReplicasToDirs:
>>> > event failed with UnsupportedVersionException in 15 microseconds.
>>> > (org.apache.kafka.controller.QuorumController)
>>> > [quorum-controller-0-event-handler]
>>> > 2024-01-13 16:58:01,197 ERROR [ControllerApis nodeId=0] Unexpected error
>>> > handling request RequestHeader(apiKey=ASSIGN_REPLICAS_TO_DIRS,
>>> > apiVersion=0, clientId=1000, correlationId=14, headerVersion=2) --
>>> > AssignReplicasToDirsRequestData(brokerId=1000, brokerEpoch=5,
>>> > directories=[DirectoryData(id=w_uxN7pwQ6eXSMrOKceYIQ,
>>> > topics=[TopicData(topicId=bvAKLSwmR7iJoKv2yZgygQ,
>>> > partitions=[PartitionData(partitionIndex=2),
>>> > PartitionData(partitionIndex=1)]),
>>> > TopicData(topicId=uNe7f5VrQgO0zST6yH1jDQ,
>>> > partitions=[PartitionData(partitionIndex=0)])])]) with context
>>> > 

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

2024-01-15 Thread Apache Jenkins Server
See