Re: [DISCUSS] KIP-1044: A proposal to change idempotent producer -- server implementation

2024-05-23 Thread Igor Soarez
Hi Claude, Thanks for writing this KIP. This issue seems particularly thorny, and I appreciate everyone's effort to address this. I want to share my concern with the KIP's proposal of the use of memory mapped files – mmap is Java's achilles heel, Kafka should make less use of it, not more. The

[jira] [Resolved] (KAFKA-16645) CVEs in 3.7.0 docker image

2024-05-20 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16645?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez resolved KAFKA-16645. - Resolution: Resolved > CVEs in 3.7.0 docker im

[jira] [Reopened] (KAFKA-16645) CVEs in 3.7.0 docker image

2024-05-20 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16645?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez reopened KAFKA-16645: - Need to re-open to change the resolution, release_notes.py doesn't like the one I picked > C

[jira] [Reopened] (KAFKA-16692) InvalidRequestException: ADD_PARTITIONS_TO_TXN with version 4 which is not enabled when upgrading from kafka 3.5 to 3.6

2024-05-20 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16692?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez reopened KAFKA-16692: - Re-opening as 3.6 backport is still missing > InvalidRequestException: ADD_PARTITIONS_TO_

[jira] [Resolved] (KAFKA-16692) InvalidRequestException: ADD_PARTITIONS_TO_TXN with version 4 which is not enabled when upgrading from kafka 3.5 to 3.6

2024-05-20 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16692?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez resolved KAFKA-16692. - Resolution: Fixed > InvalidRequestException: ADD_PARTITIONS_TO_TXN with version 4 wh

[jira] [Resolved] (KAFKA-16645) CVEs in 3.7.0 docker image

2024-05-20 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16645?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez resolved KAFKA-16645. - Assignee: Igor Soarez Resolution: Won't Fix The vulnerability has already been addressed

[jira] [Resolved] (KAFKA-16688) SystemTimer leaks resources on close

2024-05-10 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16688?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez resolved KAFKA-16688. - Resolution: Fixed > SystemTimer leaks resources on cl

[jira] [Resolved] (KAFKA-16624) Don't generate useless PartitionChangeRecord on older MV

2024-05-04 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16624?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez resolved KAFKA-16624. - Resolution: Fixed > Don't generate useless PartitionChangeRecord on older

Re: [DISCUSS] KIP-936 Throttle number of active PIDs

2024-05-01 Thread Igor Soarez
Hi Omnia, Hi Claude, Thanks for putting this KIP together. This is an important unresolved issue in Kafka, which I have witnessed several times in production. Please see my questions below: 10 Given the goal is to prevent OOMs, do we also need to limit the number of KafkaPrincipals in use? 11.

[jira] [Created] (KAFKA-16636) Flaky test - testStickyTaskAssignorLargePartitionCount – org.apache.kafka.streams.processor.internals.StreamsAssignmentScaleTest

2024-04-27 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16636: --- Summary: Flaky test - testStickyTaskAssignorLargePartitionCount – org.apache.kafka.streams.processor.internals.StreamsAssignmentScaleTest Key: KAFKA-16636 URL: https

[jira] [Created] (KAFKA-16635) Flaky test "shouldThrottleOldSegments(String).quorum=kraft" – kafka.server.ReplicationQuotasTest

2024-04-27 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16635: --- Summary: Flaky test "shouldThrottleOldSegments(String).quorum=kraft" – kafka.server.ReplicationQuotasTest Key: KAFKA-16635 URL: https://issues.apache.org/jira/browse/K

[jira] [Created] (KAFKA-16634) Flaky test - testFenceMultipleBrokers() – org.apache.kafka.controller.QuorumControllerTest

2024-04-27 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16634: --- Summary: Flaky test - testFenceMultipleBrokers() – org.apache.kafka.controller.QuorumControllerTest Key: KAFKA-16634 URL: https://issues.apache.org/jira/browse/KAFKA-16634

[jira] [Created] (KAFKA-16633) Flaky test - testDescribeExistingGroupWithNoMembers(String, String).quorum=kraft+kip848.groupProtocol=consumer – org.apache.kafka.tools.consumer.group.DescribeConsumerGr

2024-04-27 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16633: --- Summary: Flaky test - testDescribeExistingGroupWithNoMembers(String, String).quorum=kraft+kip848.groupProtocol=consumer – org.apache.kafka.tools.consumer.group.DescribeConsumerGroupTest Key

[jira] [Created] (KAFKA-16632) Flaky test testDeleteOffsetsOfStableConsumerGroupWithTopicPartition [1] Type=Raft-Isolated, MetadataVersion=3.8-IV0, Security=PLAINTEXT – org.apache.kafka.tools.consumer

2024-04-27 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16632: --- Summary: Flaky test testDeleteOffsetsOfStableConsumerGroupWithTopicPartition [1] Type=Raft-Isolated, MetadataVersion=3.8-IV0, Security=PLAINTEXT

[jira] [Created] (KAFKA-16631) Flaky test - testDeleteOffsetsOfStableConsumerGroupWithTopicOnly [1] Type=Raft-Isolated, MetadataVersion=3.8-IV0, Security=PLAINTEXT – org.apache.kafka.tools.consumer.gr

2024-04-27 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16631: --- Summary: Flaky test - testDeleteOffsetsOfStableConsumerGroupWithTopicOnly [1] Type=Raft-Isolated, MetadataVersion=3.8-IV0, Security=PLAINTEXT

[jira] [Created] (KAFKA-16630) Flaky test "testPollReturnsRecords(GroupProtocol).groupProtocol=CLASSIC" – org.apache.kafka.clients.consumer.KafkaConsumerTest

2024-04-27 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16630: --- Summary: Flaky test "testPollReturnsRecords(GroupProtocol).groupProtocol=CLASSIC" – org.apache.kafka.clients.consumer.KafkaConsumerTest Key: KAFKA-16630

Re: [ANNOUNCE] New committer: Igor Soarez

2024-04-25 Thread Igor Soarez
Thanks everyone, I'm very honoured to join! -- Igor

[DISCUSS] Apache Kafka 3.7.1 release

2024-04-25 Thread Igor Soarez
Hi everyone, I'd like to volunteer to be the release manager for a 3.7.1 release. Please keep in mind, this would be my first release, so I might have some questions, and it might also take me a bit longer to work through the release process. So I'm thinking a good target would be toward the

[jira] [Resolved] (KAFKA-16610) Replace "Map#entrySet#forEach" by "Map#forEach"

2024-04-24 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16610?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez resolved KAFKA-16610. - Resolution: Resolved > Replace "Map#entrySet#forEach" by

[jira] [Reopened] (KAFKA-16606) JBOD support in KRaft does not seem to be gated by the metadata version

2024-04-24 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16606?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez reopened KAFKA-16606: - Assignee: Igor Soarez > JBOD support in KRaft does not seem to be gated by the metad

Re: [VOTE] KIP-1031: Control offset translation in MirrorSourceConnector

2024-04-24 Thread Igor Soarez
Hi Omnia, Thanks for your answers, and I see you've updated the KIP so thanks for the changes too. +1 (binding), thanks for the KIP -- Igor

[jira] [Created] (KAFKA-16602) Flaky test – org.apache.kafka.controller.QuorumControllerTest.testBootstrapZkMigrationRecord()

2024-04-22 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16602: --- Summary: Flaky test – org.apache.kafka.controller.QuorumControllerTest.testBootstrapZkMigrationRecord() Key: KAFKA-16602 URL: https://issues.apache.org/jira/browse/KAFKA-16602

[jira] [Created] (KAFKA-16601) Flaky test – org.apache.kafka.controller.QuorumControllerMetricsIntegrationTest.testClosingQuorumControllerClosesMetrics()

2024-04-22 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16601: --- Summary: Flaky test – org.apache.kafka.controller.QuorumControllerMetricsIntegrationTest.testClosingQuorumControllerClosesMetrics() Key: KAFKA-16601 URL: https://issues.apache.org

[jira] [Created] (KAFKA-16597) Flaky test - org.apache.kafka.streams.integration.StoreQueryIntegrationTest.shouldQuerySpecificStalePartitionStoresMultiStreamThreads()

2024-04-22 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16597: --- Summary: Flaky test - org.apache.kafka.streams.integration.StoreQueryIntegrationTest.shouldQuerySpecificStalePartitionStoresMultiStreamThreads() Key: KAFKA-16597 URL: https

[jira] [Created] (KAFKA-16596) Flaky test – org.apache.kafka.clients.ClientUtilsTest.testParseAndValidateAddressesWithReverseLookup()

2024-04-22 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16596: --- Summary: Flaky test – org.apache.kafka.clients.ClientUtilsTest.testParseAndValidateAddressesWithReverseLookup() Key: KAFKA-16596 URL: https://issues.apache.org/jira/browse/KAFKA

Re: [VOTE] KIP-1031: Control offset translation in MirrorSourceConnector

2024-04-20 Thread Igor Soarez
Hi Omnia, Thanks for this KIP. 11. These seem to me to be small misspellings, please double-check: s/MM2 main features/MM2's main features s/syncing consumer group offset/syncing consumer group offsets s/relays/relies s/recored's offset/recorded offsets s/clusters without need for/clusters

[jira] [Reopened] (KAFKA-15793) Flaky test ZkMigrationIntegrationTest.testMigrateTopicDeletions

2024-04-10 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15793?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez reopened KAFKA-15793: - This has come up again:   {code:java} [2024-04-09T21:06:17.307Z] Gradle Test Run :core:test

[jira] [Created] (KAFKA-16504) Flaky test org.apache.kafka.controller.QuorumControllerTest.testConfigurationOperations

2024-04-10 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16504: --- Summary: Flaky test org.apache.kafka.controller.QuorumControllerTest.testConfigurationOperations Key: KAFKA-16504 URL: https://issues.apache.org/jira/browse/KAFKA-16504

[jira] [Resolved] (KAFKA-16403) Flaky test org.apache.kafka.streams.examples.wordcount.WordCountDemoTest.testCountListOfWords

2024-03-27 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16403?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez resolved KAFKA-16403. - Resolution: Not A Bug > Flaky t

[jira] [Resolved] (KAFKA-16404) Flaky test org.apache.kafka.streams.examples.wordcount.WordCountDemoTest.testGetStreamsConfig

2024-03-27 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16404?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez resolved KAFKA-16404. - Resolution: Not A Bug Same as KAFKA-16403, this only failed once. It was likely the result

[jira] [Created] (KAFKA-16422) Flaky test org.apache.kafka.controller.QuorumControllerMetricsIntegrationTest."testFailingOverIncrementsNewActiveControllerCount(boolean).true"

2024-03-25 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16422: --- Summary: Flaky test org.apache.kafka.controller.QuorumControllerMetricsIntegrationTest."testFailingOverIncrementsNewActiveControllerCount(boolean).true" Key: KAFKA-16422

[jira] [Created] (KAFKA-16404) Flaky test org.apache.kafka.streams.examples.wordcount.WordCountDemoTest.testGetStreamsConfig

2024-03-22 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16404: --- Summary: Flaky test org.apache.kafka.streams.examples.wordcount.WordCountDemoTest.testGetStreamsConfig Key: KAFKA-16404 URL: https://issues.apache.org/jira/browse/KAFKA-16404

[jira] [Created] (KAFKA-16403) Flaky test org.apache.kafka.streams.examples.wordcount.WordCountDemoTest.testCountListOfWords

2024-03-22 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16403: --- Summary: Flaky test org.apache.kafka.streams.examples.wordcount.WordCountDemoTest.testCountListOfWords Key: KAFKA-16403 URL: https://issues.apache.org/jira/browse/KAFKA-16403

[jira] [Created] (KAFKA-16402) Flaky test org.apache.kafka.controller.QuorumControllerTest.testSnapshotSaveAndLoad

2024-03-22 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16402: --- Summary: Flaky test org.apache.kafka.controller.QuorumControllerTest.testSnapshotSaveAndLoad Key: KAFKA-16402 URL: https://issues.apache.org/jira/browse/KAFKA-16402

[jira] [Created] (KAFKA-16365) AssignmentsManager mismanages completion notifications

2024-03-11 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16365: --- Summary: AssignmentsManager mismanages completion notifications Key: KAFKA-16365 URL: https://issues.apache.org/jira/browse/KAFKA-16365 Project: Kafka Issue

[jira] [Created] (KAFKA-16363) Storage crashes if dir is unavailable

2024-03-11 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16363: --- Summary: Storage crashes if dir is unavailable Key: KAFKA-16363 URL: https://issues.apache.org/jira/browse/KAFKA-16363 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-16297) Race condition while promoting future replica can lead to partition unavailability.

2024-02-22 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-16297: --- Summary: Race condition while promoting future replica can lead to partition unavailability. Key: KAFKA-16297 URL: https://issues.apache.org/jira/browse/KAFKA-16297

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-12-04 Thread Igor Soarez
d it make sense to authorize these requests as other inter-broker > protocol calls are usually authorized, that is ClusterAction on Cluster > resource? > > Thanks, > Viktor > > On Tue, Nov 28, 2023 at 4:18 PM Igor Soarez wrote: > > > Hi everyone, > > > > Th

[jira] [Created] (KAFKA-15955) Migrating ZK brokers send dir assignments

2023-12-01 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15955: --- Summary: Migrating ZK brokers send dir assignments Key: KAFKA-15955 URL: https://issues.apache.org/jira/browse/KAFKA-15955 Project: Kafka Issue Type: Sub-task

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-11-28 Thread Igor Soarez
is now more of an open question. It's unclear if this will actually be necessary. Please share if you have any thoughts. Best, -- Igor On Tue, Oct 10, 2023, at 5:28 AM, Igor Soarez wrote: > Hi Colin, > > Thanks for the renaming suggestions. UNASSIGNED is better then > UNKNOWN, MIGRA

[jira] [Created] (KAFKA-15893) Bump MetadataVersion for directory assignments

2023-11-24 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15893: --- Summary: Bump MetadataVersion for directory assignments Key: KAFKA-15893 URL: https://issues.apache.org/jira/browse/KAFKA-15893 Project: Kafka Issue Type: Sub

[jira] [Created] (KAFKA-15886) Always specify directories for new partition registrations

2023-11-22 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15886: --- Summary: Always specify directories for new partition registrations Key: KAFKA-15886 URL: https://issues.apache.org/jira/browse/KAFKA-15886 Project: Kafka

[jira] [Created] (KAFKA-15858) Broker stays fenced until all assignments are correct

2023-11-20 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15858: --- Summary: Broker stays fenced until all assignments are correct Key: KAFKA-15858 URL: https://issues.apache.org/jira/browse/KAFKA-15858 Project: Kafka Issue

Re: [DISCUSS] Should we continue to merge without a green build? No!

2023-11-16 Thread Igor Soarez
Hi all, I think at least one of those is my fault, apologies. I'll try to make sure all my tests are passing from now on. It doesn't help that GitHub always shows that the tests have failed, even when they have not. I suspect this is because Jenkins always marks the builds as unstable, even when

[jira] [Created] (KAFKA-15650) Data-loss on leader shutdown right after partition creation?

2023-10-19 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15650: --- Summary: Data-loss on leader shutdown right after partition creation? Key: KAFKA-15650 URL: https://issues.apache.org/jira/browse/KAFKA-15650 Project: Kafka

[jira] [Created] (KAFKA-15649) Handle directory failure timeout

2023-10-19 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15649: --- Summary: Handle directory failure timeout Key: KAFKA-15649 URL: https://issues.apache.org/jira/browse/KAFKA-15649 Project: Kafka Issue Type: Sub-task

[jira] [Reopened] (KAFKA-15355) Message schema changes

2023-10-16 Thread Igor Soarez (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15355?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Soarez reopened KAFKA-15355: - closed by mistake > Message schema changes > -- > >

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-10-06 Thread Igor Soarez
Hi Colin, > I would call #2 LOST. It was assigned in the past, but we don't know where. > I see that you called this OFFLINE). This is not really normal... > it should happen only when we're migrating from ZK mode to KRaft mode, > or going from an older KRaft release with multiple directories to

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-10-06 Thread Igor Soarez
Hi David, Thanks for shedding light on migration goals, makes sense. Your preference for option a) makes it even more attractive. We'll keep that as the preferred approach, thanks for the advice. > One question with this approach is how the KRaft controller learns about > the multiple log

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-10-04 Thread Igor Soarez
Hi everyone, Earlier today Colin, Ron, Proven and I had a chat about this work. We discussed several aspects which I’d like to share here. ## A new reserved UUID We'll reserve a third UUID to indicate an unspecified dir, but one that is known to be selected. As opposed to the default

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-09-27 Thread Igor Soarez
Hi everyone, After a conversation with Colin McCabe and Proven Provenzano yesterday, we decided that the benefits outweigh the concerns with the overhead of associating a directory UUID to every replica in the metadata partition records. i.e. We prefer to always associate the log dir UUID even

[jira] [Created] (KAFKA-15514) Controller-side replica management changes

2023-09-27 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15514: --- Summary: Controller-side replica management changes Key: KAFKA-15514 URL: https://issues.apache.org/jira/browse/KAFKA-15514 Project: Kafka Issue Type: Sub

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-09-25 Thread Igor Soarez
Hi Ron, I think we can generalize the deconfigured directory scenario in your last question to address this situation too. When handling a broker registration request, the controller can check if OfflineLogDirs=false and any UUIDs are missing in OnlineLogDirs, compared with the previous

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-09-23 Thread Igor Soarez
Hi everyone, I believe we can close this voting thread now, as there were three +1 binding votes from Ziming, Mickael and Ron. With that, this vote passes. Thanks to everyone who participated in reviewing, and/or taking the time to vote on this KIP! Best, -- Igor

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-09-23 Thread Igor Soarez
wrote: > > > > Ok, great, that makes sense, Igor. Thanks. +1 (binding) on the KIP from > > me. > > > > Ron > > > > > On Sep 13, 2023, at 11:58 AM, Igor Soarez > > > wrote: > > > > > > Hi Ron, > > > > > &

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-09-13 Thread Igor Soarez
Hi Ron, Thanks for drilling down on this. I think the KIP isn't really clear here, and the metadata caching section you quoted needs clarification. The "hosting broker's latest registration" refers to the previous, not the current registration. The registrations are only compared by the

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-09-12 Thread Igor Soarez
Hi Ron, Thank you for having a look a this KIP. Indeed, the log directory UUID should always be generated and loaded. I've have corrected the wording in the KIP to clarify. It is a bit of a pain to replace the field, but I agree that is the best approach for the same reason you pointed out. I

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-09-12 Thread Igor Soarez
Hi Ziming, Thank you for having a look and taking the time to vote. I have already opened some PRs, see: https://issues.apache.org/jira/browse/KAFKA-14127 Best, -- Igor

[jira] [Created] (KAFKA-15451) Include offline dirs in BrokerHeartbeatRequest

2023-09-11 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15451: --- Summary: Include offline dirs in BrokerHeartbeatRequest Key: KAFKA-15451 URL: https://issues.apache.org/jira/browse/KAFKA-15451 Project: Kafka Issue Type: Sub

[jira] [Created] (KAFKA-15426) Process and persist directory assignments

2023-08-31 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15426: --- Summary: Process and persist directory assignments Key: KAFKA-15426 URL: https://issues.apache.org/jira/browse/KAFKA-15426 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-15368) Test ZK JBOD to KRaft migration

2023-08-16 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15368: --- Summary: Test ZK JBOD to KRaft migration Key: KAFKA-15368 URL: https://issues.apache.org/jira/browse/KAFKA-15368 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-15367) Test KRaft JBOD enabling migration

2023-08-16 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15367: --- Summary: Test KRaft JBOD enabling migration Key: KAFKA-15367 URL: https://issues.apache.org/jira/browse/KAFKA-15367 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-15366) Log directory failure integration test

2023-08-16 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15366: --- Summary: Log directory failure integration test Key: KAFKA-15366 URL: https://issues.apache.org/jira/browse/KAFKA-15366 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-15365) Replica management changes

2023-08-16 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15365: --- Summary: Replica management changes Key: KAFKA-15365 URL: https://issues.apache.org/jira/browse/KAFKA-15365 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-15364) Handle log directory failure in the Controller

2023-08-16 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15364: --- Summary: Handle log directory failure in the Controller Key: KAFKA-15364 URL: https://issues.apache.org/jira/browse/KAFKA-15364 Project: Kafka Issue Type: Sub

[jira] [Created] (KAFKA-15363) Broker log directory failure changes

2023-08-16 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15363: --- Summary: Broker log directory failure changes Key: KAFKA-15363 URL: https://issues.apache.org/jira/browse/KAFKA-15363 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-15362) Resolve offline replicas in metadata cache

2023-08-16 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15362: --- Summary: Resolve offline replicas in metadata cache Key: KAFKA-15362 URL: https://issues.apache.org/jira/browse/KAFKA-15362 Project: Kafka Issue Type: Sub

[jira] [Created] (KAFKA-15361) Process and persist dir info with broker registration

2023-08-16 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15361: --- Summary: Process and persist dir info with broker registration Key: KAFKA-15361 URL: https://issues.apache.org/jira/browse/KAFKA-15361 Project: Kafka Issue

[jira] [Created] (KAFKA-15360) Include directory info in BrokerRegistration

2023-08-16 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15360: --- Summary: Include directory info in BrokerRegistration Key: KAFKA-15360 URL: https://issues.apache.org/jira/browse/KAFKA-15360 Project: Kafka Issue Type: Sub

[jira] [Created] (KAFKA-15359) log.dir.failure.timeout.ms configuration

2023-08-16 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15359: --- Summary: log.dir.failure.timeout.ms configuration Key: KAFKA-15359 URL: https://issues.apache.org/jira/browse/KAFKA-15359 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-15358) QueuedReplicaToDirAssignments metric

2023-08-16 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15358: --- Summary: QueuedReplicaToDirAssignments metric Key: KAFKA-15358 URL: https://issues.apache.org/jira/browse/KAFKA-15358 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-15357) Propagates assignments and logdir failures to controller

2023-08-16 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15357: --- Summary: Propagates assignments and logdir failures to controller Key: KAFKA-15357 URL: https://issues.apache.org/jira/browse/KAFKA-15357 Project: Kafka Issue

[jira] [Created] (KAFKA-15356) Generate and persist log directory UUIDs

2023-08-16 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15356: --- Summary: Generate and persist log directory UUIDs Key: KAFKA-15356 URL: https://issues.apache.org/jira/browse/KAFKA-15356 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-15355) Update metadata records

2023-08-16 Thread Igor Soarez (Jira)
Igor Soarez created KAFKA-15355: --- Summary: Update metadata records Key: KAFKA-15355 URL: https://issues.apache.org/jira/browse/KAFKA-15355 Project: Kafka Issue Type: Sub-task

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-08-10 Thread Igor Soarez
Hi Mickael, Thanks for voting, and for pointing out the mistake. I've corrected it in the KIP now. The proposed name is "QueuedReplicaToDirAssignments". Best, -- Igor

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-07-25 Thread Igor Soarez
Hi Ismael, I believe I have addressed all concerns. Please have a look, and consider a vote on this KIP. Thank you, -- Igor

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-07-25 Thread Igor Soarez
Hi everyone, Following a face-to-face discussion with Ron and Colin, I have just made further improvements to this KIP: 1. Every log directory gets a random UUID assigned, even if just one log dir is configured in the Broker. 2. All online log directories are registered, even if just one if

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-07-10 Thread Igor Soarez
Hi Colin, Thanks for your questions. Please have a look at my answers below. > In the previous email I asked, "who is responsible for assigning replicas to > broker directories?" Can you clarify what the answer is to that? If the > answer is the controller, there is no need for an "unknown"

Re: [VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-06-26 Thread Igor Soarez
Hi Colin, Thanks for your support with getting this over the line and that’s great re the preliminary pass! Thanks also for sharing your thoughts, I've had a careful look at each of these and sharing my comments below. I agree, it is important to avoid a perf hit on non-JBOD. I've opted for

Re: [ANNOUNCE] New committer: Divij Vaidya

2023-06-13 Thread Igor Soarez
Congratulations Divij! -- Igor

[VOTE] KIP-858: Handle JBOD broker disk failure in KRaft

2023-06-12 Thread Igor Soarez
Hi everyone, We're getting closer to dropping ZooKeeper support, and JBOD in KRaft mode is one of the outstanding big missing features. It's been a while since there was new feedback on KIP-858 [1] which aims to address this gap, so I'm calling for a vote. A huge thank you to everyone who has

Re: [VOTE] KIP-938: Add more metrics for measuring KRaft performance

2023-06-06 Thread Igor Soarez
Thanks for the KIP. Seems straightforward, LGTM. Non binding +1. -- Igor

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2023-06-05 Thread Igor Soarez
Hi all, We just had a video call to discuss this KIP and I just wanted update this thread with a note on the meeting. Attendees: - Igor - Christo - Divij - Colt Items discussed: - Context, motivation and overview of the proposal. - How log directories are identified by each Broker. - How old

Re: [DISCUSS] KIP-928: Making Kafka resilient to log directories becoming full

2023-06-02 Thread Igor Soarez
Hi Christo, Thank you for the KIP. Kafka is very sensitive to filesystem errors, and at the first IO error the whole log directory is permanently considered offline. It seems your proposal aims to increase the robustness of Kafka, and that's a positive improvement. I have some questions: 11.

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2023-05-31 Thread Igor Soarez
Hi all, I have created a TLA+ specification for this KIP, available here: https://github.com/soarez/kafka/blob/kip-858-tla-plus/tla/Kip858.tla If there are no further comments I'll start a voting thread next week. -- Igor

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2023-05-30 Thread Igor Soarez
Hi Alexandre, Thank you for having a look at this KIP, and thank you for pointing this out. I like the idea of expanding the health status of a log directory beyond just online/offline status. This KIP currently proposes a single logdir state transition, from online to offline, conveyed in a

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2023-05-25 Thread Igor Soarez
Hi Divij, Christo, Thank you for pointing that out. Let's aim instead for Monday 5th of June, at the same time – 16:30-17:00 UTC. Please let me know if this doesn't work either. Best, -- Igor

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2023-05-23 Thread Igor Soarez
Hi everyone, Someone suggested at the recent Kafka Summit that it may be useful to have a video call to discuss remaining concerns. I'm proposing we have a video call Monday 29th May 16:30-17:00 UTC. If you'd like to join, please reply to the thread or to me directly so I can send you a link.

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2023-05-23 Thread Igor Soarez
Hi Christo, Thank you for your interest in this KIP. Indeed, I'd like to open up voting ASAP. I'm hoping there will still be a bit more feedback, but if not I'll probably request a vote next week or so. Do you have any concerns or suggestions regarding this KIP? I'll have a look at your KIP

Re: [DISCUSS] Re-visit end of life policy

2023-05-16 Thread Igor Soarez
My impression is also that a lot of users run older, out of EOL, versions of Kafka. The final 3.x version is particularly concerning, as it will be the last bridge to migrate away from ZK. If a big portion of users only upgrade after its EOL period, we might only then discover an important bug

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2023-04-26 Thread Igor Soarez
Thank you for another review Ziming, much appreciated! 1. and 2. You are correct, it would be a big and perhaps strange difference. Since our last exchange of emails, the proposal has changed and now it does follow your suggestion to bump metadata.version. The KIP mentions it under

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2023-04-17 Thread Igor Soarez
Hi Jun, Thank you for sharing your questions, please find my answers below. 41. There can only be user partitions on `metadata.log.dir` if that log dir is also listed in `log.dirs`. `LogManager` does not specifically load contents from `metadata.log.dir`. The broker will communicate UUIDs to

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2023-02-24 Thread Igor Soarez
Hi all, I’ve had to step away from work for personal reasons for a couple of months – until mid April 2023. I don’t think I’ll be able to continue to address feedback or update this KIP before then. -- Igor

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2023-02-06 Thread Igor Soarez
Hi David, Thank you for your suggestions and for having a look at this KIP. 1. Yes, that should be OK. I have updated the section "Migrating a cluster in ZK mode running with JBOD" to reflect this. 2. I've updated the motivation section to state that. Best, -- Igor

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2023-02-03 Thread Igor Soarez
Hi Jun, Thank you for your comments and questions. 30. Thank you for pointing this out. The isNew flag is not available in KRaft mode. The broker can consider the metadata records: If, and only if, the logdir assigned is Uuid.ZERO then the replica can be considered new. Being able to determine

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2023-02-03 Thread Igor Soarez
Hi Tom, Thank you for having another look. 20. That is a good point. Thinking about your suggestion: How would this look like in a non-JBOD KRraft cluster upgrade to JBOD mode? Upgrading to version that includes the JBOD support patch would automatically update meta.properties to include the

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2023-01-13 Thread Igor Soarez
Hi Tom, Thank you for having another look. 20. Upon a downgrade to a Kafka version that runs the current "version == 1" assertion, then yes — a downgrade would not be possible without first updating (manually) the meta.properties files back to the previous version. We could prevent this issue

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2023-01-03 Thread Igor Soarez
Hi Jun, Thank you for having another look. 11. That is correct. I have updated the KIP in an attempt to make this clearer. I think the goal should be to try to minimize the chance that a log directory may happen while the metadata is incorrect about the log directory assignment, but also have a

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2022-12-01 Thread Igor Soarez
Hi Jun, Thank you for reviewing the KIP. Please find my replies to your comments below. 10. Thanks for pointing out this typo; it has been corrected. 11. I agree that the additional delay in switching to the future replica is undesirable, however I see a couple of issues if we forward the

Re: [DISCUSS] KIP-866 ZooKeeper to KRaft Migration

2022-11-24 Thread Igor Soarez
Hi David, Zookeeper mode writes meta.properties with version=0. KRaft mode requires version=1 in meta.properties. Will a manual step be required to update meta.properties or will brokers somehow update meta.properties files to version 1? Thanks, -- Igor

  1   2   >