[jira] [Created] (KAFKA-16667) KRaftMigrationDriver gets stuck after successive failovers

2024-05-04 Thread David Arthur (Jira)
David Arthur created KAFKA-16667:


 Summary: KRaftMigrationDriver gets stuck after successive failovers
 Key: KAFKA-16667
 URL: https://issues.apache.org/jira/browse/KAFKA-16667
 Project: Kafka
  Issue Type: Bug
  Components: controller, migration
Reporter: David Arthur


This is a continuation of KAFKA-16171.

It turns out that the active KRaftMigrationDriver can get a stale read from ZK 
after becoming the active controller in ZK (i.e., writing to "/controller").

Because ZooKeeper only offers linearizability on writes to a given ZNode, it is 
possible that we get a stale read on the "/migration" ZNode after writing to 
"/controller" (and "/controller_epoch") when becoming active. 

 

The history looks like this:
 # Node B becomes leader in the Raft layer. KRaftLeaderEvents are enqueued on 
all KRaftMigrationDriver-s
 # Node A writes some state to ZK, updates "/migration", and checks 
"/controller_epoch" in one transaction. This happens before B claims controller 
leadership in ZK. The "/migration" state is updated from X to Y
 # Node B claims leadership by updating "/controller" and "/controller_epoch". 
Leader B reads "/migration" state X
 # Node A tries to write some state, fails on "/controller_epoch" check op.
 # Node A processes new leader and becomes inactive

 

This does not violate consistency guarantees made by ZooKeeper.

 

> Write operations in ZooKeeper are {_}linearizable{_}. In other words, each 
> {{write}} will appear to take effect atomically at some point between when 
> the client issues the request and receives the corresponding response.

and 

> Read operations in ZooKeeper are _not linearizable_ since they can return 
> potentially stale data. This is because a {{read}} in ZooKeeper is not a 
> quorum operation and a server will respond immediately to a client that is 
> performing a {{{}read{}}}.

 

--- 

 

The impact of this stale read is the same as KAFKA-16171. The 
KRaftMigrationDriver never gets past SYNC_KRAFT_TO_ZK because it has a stale 
zkVersion for the "/migration" ZNode. The result is brokers never learn about 
the new controller and cannot update any partition state.

The workaround for this bug is to re-elect the controller by shutting down the 
active KRaft controller. 

This bug was found during a migration where the KRaft controller was rapidly 
failing over due to an excess of metadata. 



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


[jira] [Created] (KAFKA-16539) Can't update specific broker configs in pre-migration mode

2024-04-11 Thread David Arthur (Jira)
David Arthur created KAFKA-16539:


 Summary: Can't update specific broker configs in pre-migration mode
 Key: KAFKA-16539
 URL: https://issues.apache.org/jira/browse/KAFKA-16539
 Project: Kafka
  Issue Type: Bug
  Components: config, kraft
Affects Versions: 3.6.2, 3.6.1, 3.7.0, 3.6.0
Reporter: David Arthur
Assignee: David Arthur
 Fix For: 3.8.0, 3.7.1, 3.6.3


In migration mode, ZK brokers will have a forwarding manager configured. This 
is used to forward requests to the KRaft controller once we get to that part of 
the migration. However, prior to KRaft taking over as the controller (known as 
pre-migration mode), the ZK brokers are still attempting to forward 
IncrementalAlterConfigs to the controller.

This works fine for cluster level configs (e.g., "--entity-type broker 
--entity-default"), but this fails for specific broker configs (e.g., 
"--entity-type broker --entity-id 1").

This affects BROKER and BROKER_LOGGER config types.

To workaround this bug, you can either disable migrations on the brokers 
(assuming no migration has taken place), or proceed with the migration and get 
to the point where KRaft is the controller.



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


[jira] [Created] (KAFKA-16468) Listener not found error in SendRPCsToBrokersEvent

2024-04-03 Thread David Arthur (Jira)
David Arthur created KAFKA-16468:


 Summary: Listener not found error in SendRPCsToBrokersEvent
 Key: KAFKA-16468
 URL: https://issues.apache.org/jira/browse/KAFKA-16468
 Project: Kafka
  Issue Type: Bug
  Components: controller, migration
Reporter: David Arthur
 Fix For: 3.8.0


During the ZK to KRaft migration, the controller will send RPCs using the 
configured "control.plane.listener.name" or more commonly, the 
"inter.broker.listener.name". If a ZK broker did not register with this 
listener, we get a error at the time of sending the first RPC to a broker.

{code}
[2024-04-03 09:28:59,043] ERROR Encountered nonFatalFaultHandler fault: 
Unhandled error in SendRPCsToBrokersEvent 
(org.apache.kafka.server.fault.MockFaultHandler:44)
kafka.common.BrokerEndPointNotAvailableException: End point with listener name 
EXTERNAL not found for broker 0
at kafka.cluster.Broker.$anonfun$node$1(Broker.scala:94)
at scala.Option.getOrElse(Option.scala:201)
at kafka.cluster.Broker.node(Broker.scala:93)
at 
kafka.controller.ControllerChannelManager.addNewBroker(ControllerChannelManager.scala:122)
at 
kafka.controller.ControllerChannelManager.addBroker(ControllerChannelManager.scala:105)
at 
kafka.migration.MigrationPropagator.$anonfun$publishMetadata$2(MigrationPropagator.scala:98)
at 
kafka.migration.MigrationPropagator.$anonfun$publishMetadata$2$adapted(MigrationPropagator.scala:98)
at scala.collection.immutable.Set$Set3.foreach(Set.scala:261)
at 
kafka.migration.MigrationPropagator.publishMetadata(MigrationPropagator.scala:98)
at 
kafka.migration.MigrationPropagator.sendRPCsToBrokersFromMetadataImage(MigrationPropagator.scala:219)
at 
org.apache.kafka.metadata.migration.KRaftMigrationDriver$SendRPCsToBrokersEvent.run(KRaftMigrationDriver.java:777)
at 
org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:128)
at 
org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:211)
at 
org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:182)
at java.base/java.lang.Thread.run(Thread.java:833)
{code}

At this point, the KRaft controller has already migrated the metadata. Recovery 
at this point is possible by restarting the brokers with the correct listener 
names, but we can catch this much sooner in the process.

When a ZK broker registers with the KRaft controller, we should reject the 
registration if the expected listener name is not present. This will prevent 
the migration from starting.



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


[jira] [Created] (KAFKA-16466) QuorumController is swallowing some exception messages

2024-04-02 Thread David Arthur (Jira)
David Arthur created KAFKA-16466:


 Summary: QuorumController is swallowing some exception messages
 Key: KAFKA-16466
 URL: https://issues.apache.org/jira/browse/KAFKA-16466
 Project: Kafka
  Issue Type: Bug
  Components: controller
Affects Versions: 3.7.0
Reporter: David Arthur
 Fix For: 3.8.0, 3.7.1


In some cases in QuorumController, we throw exceptions from the control manager 
methods. Unless these are explicitly caught and handled, they will eventually 
bubble up to the ControllerReadEvent/ControllerWriteEvent an hit the generic 
error handler.

In the generic error handler of QuorumController, we examine the exception to 
determine if it is a fault or not. In the case where it is not a fault, we log 
the error like:
{code:java}
 log.info("{}: {}", name, failureMessage);
{code}
which results in messages like
{code:java}
[2024-04-02 16:08:38,078] INFO [QuorumController id=3000] registerBroker: event 
failed with UnsupportedVersionException in 167 microseconds. 
(org.apache.kafka.controller.QuorumController:544)
{code}
In this case, the exception actually has more details in its own message
{code:java}
Unable to register because the broker does not support version 8 of 
metadata.version. It wants a version between 20 and 20, inclusive.
{code}

This was found while writing an integration test for KRaft migration where the 
brokers and controllers have a mismatched MetadataVersion.



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


[jira] [Created] (KAFKA-16463) Automatically delete metadata log directory on ZK brokers

2024-04-02 Thread David Arthur (Jira)
David Arthur created KAFKA-16463:


 Summary: Automatically delete metadata log directory on ZK brokers
 Key: KAFKA-16463
 URL: https://issues.apache.org/jira/browse/KAFKA-16463
 Project: Kafka
  Issue Type: Improvement
Reporter: David Arthur
Assignee: David Arthur
 Fix For: 3.8.0


Throughout the process of a ZK to KRaft migration, the operator has the choice 
to revert back to ZK mode. Once this is done, there will be a copy of the 
metadata log on each broker in the cluster.

In order to re-attempt the migration in the future, this metadata log needs to 
be deleted. This can be pretty burdensome to the operator for large clusters. 

To improve this, we can automatically delete any metadata log present during 
startup of a ZK broker. This is safe to do because the ZK broker will just 
re-replicate the metadata log from the active controller.



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


[jira] [Created] (KAFKA-16446) Log slow controller events

2024-03-28 Thread David Arthur (Jira)
David Arthur created KAFKA-16446:


 Summary: Log slow controller events
 Key: KAFKA-16446
 URL: https://issues.apache.org/jira/browse/KAFKA-16446
 Project: Kafka
  Issue Type: Improvement
Reporter: David Arthur


Occasionally, we will see very high p99 controller event processing times. 
Unless DEBUG logs are enabled, it is impossible to see which events are slow. 

Typically this happens during controller startup/failover, though it can also 
happen sporadically when the controller gets overloaded.



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


[jira] [Resolved] (KAFKA-16180) Full metadata request sometimes fails during zk migration

2024-03-14 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-16180.
--
Resolution: Fixed

> Full metadata request sometimes fails during zk migration
> -
>
> Key: KAFKA-16180
> URL: https://issues.apache.org/jira/browse/KAFKA-16180
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.7.0
>Reporter: Colin McCabe
>Priority: Blocker
> Fix For: 3.6.2, 3.7.0
>
>
> Example:
> {code:java}
> java.util.NoSuchElementException: topic_name
> at 
> scala.collection.mutable.AnyRefMap$ExceptionDefault.apply(AnyRefMap.scala:508)
> at 
> scala.collection.mutable.AnyRefMap$ExceptionDefault.apply(AnyRefMap.scala:507)
> at scala.collection.mutable.AnyRefMap.apply(AnyRefMap.scala:207)
> at 
> kafka.server.metadata.ZkMetadataCache$.$anonfun$maybeInjectDeletedPartitionsFromFullMetadataRequest$2(ZkMetadataCache.scala:112)
> at 
> kafka.server.metadata.ZkMetadataCache$.$anonfun$maybeInjectDeletedPartitionsFromFullMetadataRequest$2$adapted(ZkMetadataCache.scala:105)
> at scala.collection.immutable.HashSet.foreach(HashSet.scala:958)
> at 
> kafka.server.metadata.ZkMetadataCache$.maybeInjectDeletedPartitionsFromFullMetadataRequest(ZkMetadataCache.scala:105)
> at 
> kafka.server.metadata.ZkMetadataCache.$anonfun$updateMetadata$1(ZkMetadataCache.scala:506)
> at kafka.utils.CoreUtils$.inWriteLock(CoreUtils.scala:183)
> at 
> kafka.server.metadata.ZkMetadataCache.updateMetadata(ZkMetadataCache.scala:496)
> at 
> kafka.server.ReplicaManager.maybeUpdateMetadataCache(ReplicaManager.scala:2482)
> at 
> kafka.server.KafkaApis.handleUpdateMetadataRequest(KafkaApis.scala:733)
> at kafka.server.KafkaApis.handle(KafkaApis.scala:349)
> at 
> kafka.server.KafkaRequestHandler.$anonfun$poll$8(KafkaRequestHandler.scala:210)
> at 
> kafka.server.KafkaRequestHandler.$anonfun$poll$8$adapted(KafkaRequestHandler.scala:210)
> at 
> io.confluent.kafka.availability.ThreadCountersManager.wrapEngine(ThreadCountersManager.java:146)
> at 
> kafka.server.KafkaRequestHandler.poll(KafkaRequestHandler.scala:210)
> at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:151)
> at java.base/java.lang.Thread.run(Thread.java:1583)
> at org.apache.kafka.common.utils.KafkaThread.run(KafkaThread.java:66)
> {code}



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


[jira] [Resolved] (KAFKA-16171) Controller failover during ZK migration can prevent metadata updates to ZK brokers

2024-03-13 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-16171.
--
Resolution: Fixed

> Controller failover during ZK migration can prevent metadata updates to ZK 
> brokers
> --
>
> Key: KAFKA-16171
> URL: https://issues.apache.org/jira/browse/KAFKA-16171
> Project: Kafka
>  Issue Type: Bug
>  Components: controller, kraft, migration
>Affects Versions: 3.6.0, 3.7.0, 3.6.1
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Blocker
> Fix For: 3.6.2, 3.7.0
>
>
> h2. Description
> During the ZK migration, after KRaft becomes the active controller we enter a 
> state called hybrid mode. This means we have a mixture of ZK and KRaft 
> brokers. The KRaft controller updates the ZK brokers using the deprecated 
> controller RPCs (LeaderAndIsr, UpdateMetadata, etc). 
>  
> A race condition exists where the KRaft controller will get stuck in a retry 
> loop while initializing itself after a failover which prevents it from 
> sending these RPCs to ZK brokers.
> h2. Impact
> Since the KRaft controller cannot send any RPCs to the ZK brokers, the ZK 
> brokers will not receive any metadata updates. The ZK brokers will be able to 
> send requests to the controller (such as AlterPartitions), but the metadata 
> updates which come as a result of those requests will never be seen. This 
> essentially looks like the controller is unavailable from the ZK brokers 
> perspective.
> h2. Detection and Mitigation
> This bug can be seen by observing failed ZK writes from a recently elected 
> controller.
> The tell-tale error message is:
> {code:java}
> Check op on KRaft Migration ZNode failed. Expected zkVersion = 507823. This 
> indicates that another KRaft controller is making writes to ZooKeeper. {code}
> with a stacktrace like:
> {noformat}
> java.lang.RuntimeException: Check op on KRaft Migration ZNode failed. 
> Expected zkVersion = 507823. This indicates that another KRaft controller is 
> making writes to ZooKeeper.
>   at 
> kafka.zk.KafkaZkClient.handleUnwrappedMigrationResult$1(KafkaZkClient.scala:2613)
>   at 
> kafka.zk.KafkaZkClient.unwrapMigrationResponse$1(KafkaZkClient.scala:2639)
>   at 
> kafka.zk.KafkaZkClient.$anonfun$retryMigrationRequestsUntilConnected$2(KafkaZkClient.scala:2664)
>   at 
> scala.collection.StrictOptimizedIterableOps.map(StrictOptimizedIterableOps.scala:100)
>   at 
> scala.collection.StrictOptimizedIterableOps.map$(StrictOptimizedIterableOps.scala:87)
>   at scala.collection.mutable.ArrayBuffer.map(ArrayBuffer.scala:43)
>   at 
> kafka.zk.KafkaZkClient.retryMigrationRequestsUntilConnected(KafkaZkClient.scala:2664)
>   at 
> kafka.zk.migration.ZkTopicMigrationClient.$anonfun$createTopic$1(ZkTopicMigrationClient.scala:158)
>   at 
> kafka.zk.migration.ZkTopicMigrationClient.createTopic(ZkTopicMigrationClient.scala:141)
>   at 
> org.apache.kafka.metadata.migration.KRaftMigrationZkWriter.lambda$handleTopicsSnapshot$27(KRaftMigrationZkWriter.java:441)
>   at 
> org.apache.kafka.metadata.migration.KRaftMigrationDriver.applyMigrationOperation(KRaftMigrationDriver.java:262)
>   at 
> org.apache.kafka.metadata.migration.KRaftMigrationDriver.access$300(KRaftMigrationDriver.java:64)
>   at 
> org.apache.kafka.metadata.migration.KRaftMigrationDriver$SyncKRaftMetadataEvent.lambda$run$0(KRaftMigrationDriver.java:791)
>   at 
> org.apache.kafka.metadata.migration.KRaftMigrationDriver.lambda$countingOperationConsumer$6(KRaftMigrationDriver.java:880)
>   at 
> org.apache.kafka.metadata.migration.KRaftMigrationZkWriter.lambda$handleTopicsSnapshot$28(KRaftMigrationZkWriter.java:438)
>   at java.base/java.lang.Iterable.forEach(Iterable.java:75)
>   at 
> org.apache.kafka.metadata.migration.KRaftMigrationZkWriter.handleTopicsSnapshot(KRaftMigrationZkWriter.java:436)
>   at 
> org.apache.kafka.metadata.migration.KRaftMigrationZkWriter.handleSnapshot(KRaftMigrationZkWriter.java:115)
>   at 
> org.apache.kafka.metadata.migration.KRaftMigrationDriver$SyncKRaftMetadataEvent.run(KRaftMigrationDriver.java:790)
>   at 
> org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:127)
>   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:1583)
>   at 
> org.apache.kafka.common.utils.KafkaThread.run(KafkaThread.java:66){noformat}
> To mitigate this problem, a new KRaft controller should be elected. This can 
> be done by restarting the problematic active controller. To verify that the 
> new 

[jira] [Created] (KAFKA-16206) ZkConfigMigrationClient tries to delete topic configs twice

2024-01-29 Thread David Arthur (Jira)
David Arthur created KAFKA-16206:


 Summary: ZkConfigMigrationClient tries to delete topic configs 
twice
 Key: KAFKA-16206
 URL: https://issues.apache.org/jira/browse/KAFKA-16206
 Project: Kafka
  Issue Type: Bug
  Components: migration, kraft
Reporter: David Arthur


When deleting a topic, we see spurious ERROR logs from 
kafka.zk.migration.ZkConfigMigrationClient:
 
{code:java}
Did not delete ConfigResource(type=TOPIC, name='xxx') since the node did not 
exist. {code}

This seems to happen because ZkTopicMigrationClient#deleteTopic is deleting the 
topic, partitions, and config ZNodes in one shot. Subsequent calls from 
KRaftMigrationZkWriter to delete the config encounter a NO_NODE since the ZNode 
is already gone.




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


[jira] [Created] (KAFKA-16205) Reduce number of metadata requests during hybrid mode

2024-01-29 Thread David Arthur (Jira)
David Arthur created KAFKA-16205:


 Summary: Reduce number of metadata requests during hybrid mode
 Key: KAFKA-16205
 URL: https://issues.apache.org/jira/browse/KAFKA-16205
 Project: Kafka
  Issue Type: Improvement
  Components: controller, kraft
Affects Versions: 3.6.0, 3.5.0, 3.4.0, 3.7.0
Reporter: David Arthur


When migrating a cluster with a high number of brokers and partitions, it is 
possible for the controller channel manager queue to get backed up. This can 
happen when many small RPCs are generated in response to several small 
MetadataDeltas being handled MigrationPropagator.

 

In the ZK controller, various optimizations have been made over the years to 
reduce the number of UMR and LISR sent during controlled shutdown or other 
large metadata events. For the ZK to KRaft migration, we use the MetadataLoader 
infrastructure to learn about and propagate metadata to ZK brokers.

 

We need to improve the batching in MigrationPropagator to avoid performance 
issues during the migration of large clusters.



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


[jira] [Created] (KAFKA-16171) Controller failover during ZK migration can lead to controller unavailability for ZK brokers

2024-01-19 Thread David Arthur (Jira)
David Arthur created KAFKA-16171:


 Summary: Controller failover during ZK migration can lead to 
controller unavailability for ZK brokers
 Key: KAFKA-16171
 URL: https://issues.apache.org/jira/browse/KAFKA-16171
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur
Assignee: David Arthur


h2. Description

During the ZK migration, after KRaft becomes the active controller we enter a 
state called hybrid mode. This means we have a mixture of ZK and KRaft brokers. 
The KRaft controller updates the ZK brokers using the deprecated controller 
RPCs (LeaderAndIsr, UpdateMetadata, etc). 

 

A race condition exists where the KRaft controller will get stuck in a retry 
loop while initializing itself after a failover which prevents it from sending 
these RPCs to ZK brokers.
h2. Impact

Since the KRaft controller cannot send any RPCs to the ZK brokers, the ZK 
brokers will not receive any metadata updates. The ZK brokers will be able to 
send requests to the controller (such as AlterPartitions), but the metadata 
updates which come as a result of those requests will never be seen. 
h2. Detection and Mitigation

This bug can be seen by observing failed ZK writes from a recently elected 
controller.

The tell-tale error message is:
{code:java}
Check op on KRaft Migration ZNode failed. Expected zkVersion = 507823. This 
indicates that another KRaft controller is making writes to ZooKeeper. {code}
with a stacktrace like:
{noformat}
java.lang.RuntimeException: Check op on KRaft Migration ZNode failed. Expected 
zkVersion = 507823. This indicates that another KRaft controller is making 
writes to ZooKeeper.
at 
kafka.zk.KafkaZkClient.handleUnwrappedMigrationResult$1(KafkaZkClient.scala:2613)
at 
kafka.zk.KafkaZkClient.unwrapMigrationResponse$1(KafkaZkClient.scala:2639)
at 
kafka.zk.KafkaZkClient.$anonfun$retryMigrationRequestsUntilConnected$2(KafkaZkClient.scala:2664)
at 
scala.collection.StrictOptimizedIterableOps.map(StrictOptimizedIterableOps.scala:100)
at 
scala.collection.StrictOptimizedIterableOps.map$(StrictOptimizedIterableOps.scala:87)
at scala.collection.mutable.ArrayBuffer.map(ArrayBuffer.scala:43)
at 
kafka.zk.KafkaZkClient.retryMigrationRequestsUntilConnected(KafkaZkClient.scala:2664)
at 
kafka.zk.migration.ZkTopicMigrationClient.$anonfun$createTopic$1(ZkTopicMigrationClient.scala:158)
at 
kafka.zk.migration.ZkTopicMigrationClient.createTopic(ZkTopicMigrationClient.scala:141)
at 
org.apache.kafka.metadata.migration.KRaftMigrationZkWriter.lambda$handleTopicsSnapshot$27(KRaftMigrationZkWriter.java:441)
at 
org.apache.kafka.metadata.migration.KRaftMigrationDriver.applyMigrationOperation(KRaftMigrationDriver.java:262)
at 
org.apache.kafka.metadata.migration.KRaftMigrationDriver.access$300(KRaftMigrationDriver.java:64)
at 
org.apache.kafka.metadata.migration.KRaftMigrationDriver$SyncKRaftMetadataEvent.lambda$run$0(KRaftMigrationDriver.java:791)
at 
org.apache.kafka.metadata.migration.KRaftMigrationDriver.lambda$countingOperationConsumer$6(KRaftMigrationDriver.java:880)
at 
org.apache.kafka.metadata.migration.KRaftMigrationZkWriter.lambda$handleTopicsSnapshot$28(KRaftMigrationZkWriter.java:438)
at java.base/java.lang.Iterable.forEach(Iterable.java:75)
at 
org.apache.kafka.metadata.migration.KRaftMigrationZkWriter.handleTopicsSnapshot(KRaftMigrationZkWriter.java:436)
at 
org.apache.kafka.metadata.migration.KRaftMigrationZkWriter.handleSnapshot(KRaftMigrationZkWriter.java:115)
at 
org.apache.kafka.metadata.migration.KRaftMigrationDriver$SyncKRaftMetadataEvent.run(KRaftMigrationDriver.java:790)
at 
org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:127)
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:1583)
at 
org.apache.kafka.common.utils.KafkaThread.run(KafkaThread.java:66){noformat}
To mitigate this problem, a new KRaft controller should be elected. This can be 
done by restarting the problematic active controller. To verify that the new 
controller does not encounter the race condition, look for 
{code:java}
[KRaftMigrationDriver id=9991] 9991 transitioning from SYNC_KRAFT_TO_ZK to 
KRAFT_CONTROLLER_TO_BROKER_COMM state {code}
 
h2. Details

Controller A loses leadership via Raft event (e.g., from a timeout in the Raft 
layer). A KRaftLeaderEvent is added to KRaftMigrationDriver event queue behind 
any pending MetadataChangeEvents. 

 

Controller B is elected and a KRaftLeaderEvent is added to 
KRaftMigrationDriver's queue. Since this controller is inactive, it processes 
the event immediately. 

[jira] [Created] (KAFKA-16078) InterBrokerProtocolVersion defaults to non-production MetadataVersion

2024-01-03 Thread David Arthur (Jira)
David Arthur created KAFKA-16078:


 Summary: InterBrokerProtocolVersion defaults to non-production 
MetadataVersion
 Key: KAFKA-16078
 URL: https://issues.apache.org/jira/browse/KAFKA-16078
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur
Assignee: David Arthur






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


[jira] [Created] (KAFKA-16020) Time#waitForFuture should tolerate nanosecond overflow

2023-12-15 Thread David Arthur (Jira)
David Arthur created KAFKA-16020:


 Summary: Time#waitForFuture should tolerate nanosecond overflow
 Key: KAFKA-16020
 URL: https://issues.apache.org/jira/browse/KAFKA-16020
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur


Reported by [~jsancio] here 
https://github.com/apache/kafka/pull/15007#discussion_r1428359211

Time#waitForFuture should follow the JDK recommendation for comparing elapsed 
nanoseconds to a duration.

https://docs.oracle.com/en/java/javase/21/docs/api/java.base/java/lang/System.html#nanoTime()

{quote}
For example, to measure how long some code takes to execute:

 
 long startTime = System.nanoTime();
 // ... the code being measured ...
 long elapsedNanos = System.nanoTime() - startTime;
To compare elapsed time against a timeout, use

 
 if (System.nanoTime() - startTime >= timeoutNanos) ...
instead of
 
 if (System.nanoTime() >= startTime + timeoutNanos) ...
because of the possibility of numerical overflow.
{quote}



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


[jira] [Created] (KAFKA-16007) ZK migrations can be slow for large clusters

2023-12-13 Thread David Arthur (Jira)
David Arthur created KAFKA-16007:


 Summary: ZK migrations can be slow for large clusters
 Key: KAFKA-16007
 URL: https://issues.apache.org/jira/browse/KAFKA-16007
 Project: Kafka
  Issue Type: Improvement
  Components: controller, kraft
Reporter: David Arthur
Assignee: David Arthur
 Fix For: 3.7.0, 3.6.2


On a large cluster with many single-partition topics, the ZK to KRaft migration 
took nearly half an hour:

{code}
[KRaftMigrationDriver id=9990] Completed migration of metadata from ZooKeeper 
to KRaft. 157396 records were generated in 2245862 ms across 67132 batches. The 
record types were {TOPIC_RECORD=66282, PARTITION_RECORD=72067, 
CONFIG_RECORD=17116, PRODUCER_IDS_RECORD=1, ACCESS_CONTROL_ENTRY_RECORD=1930}. 
The current metadata offset is now 332267 with an epoch of 19. Saw 36 brokers 
in the migrated metadata [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 
16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35].
{code}

This is a result of how we generate batches of records when traversing the ZK 
tree. Since we now using metadata transactions for the migration, we can 
re-batch these without any consistency problems.



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


[jira] [Created] (KAFKA-15968) QuorumController does not treat CorruptRecordException as fatal

2023-12-04 Thread David Arthur (Jira)
David Arthur created KAFKA-15968:


 Summary: QuorumController does not treat CorruptRecordException as 
fatal
 Key: KAFKA-15968
 URL: https://issues.apache.org/jira/browse/KAFKA-15968
 Project: Kafka
  Issue Type: Bug
Affects Versions: 3.6.0, 3.7.0
Reporter: David Arthur


When QuorumController encounters a CorruptRecordException, it does not include 
the exception in the log message. Since CorruptRecordException extends 
ApiException, it gets caught by the first condition in 
EventHandlerExceptionInfo#fromInternal.

The controller treats ApiException as an excepted case (for things like authz 
errors of creating a topic that already exists) so it does not cause a 
failover. If the active controller sees a corrupt record, it should be a fatal 
error.

While we are fixing this, we should audit the subclasses of ApiException and 
make sure we are handling the fatal ones correctly.



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


[jira] [Resolved] (KAFKA-15825) KRaft controller writes empty state to ZK after migration

2023-11-14 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-15825.
--
Resolution: Fixed

This bug was fixed as part of KAFKA-15605

> KRaft controller writes empty state to ZK after migration
> -
>
> Key: KAFKA-15825
> URL: https://issues.apache.org/jira/browse/KAFKA-15825
> Project: Kafka
>  Issue Type: Bug
>  Components: controller
>Affects Versions: 3.6.0
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Major
> Fix For: 3.7.0, 3.6.1
>
>
> Immediately following the ZK migration, there is a race condition where the 
> KRaftMigrationDriver can use an empty MetadataImage when performing the full 
> "SYNC_KRAFT_TO_ZK" reconciliation. 
> After the next controller failover, or when the controller loads a metadata 
> snapshot, the correct state will be written to ZK. 
> The symptom of this bug is that we see the migration complete, and then all 
> the metadata removed from ZK. For example, 
> {code}
> [KRaftMigrationDriver id=9990] Completed migration of metadata from ZooKeeper 
> to KRaft. 573 records were generated in 2204 ms across 51 batches. The record 
> types were {TOPIC_RECORD=41, PARTITION_RECORD=410, CONFIG_RECORD=121, 
> PRODUCER_IDS_RECORD=1}. The current metadata offset is now 503794 with an 
> epoch of 21. Saw 6 brokers in the migrated metadata [0, 1, 2, 3, 4, 5].
> {code}
> immediately followed by:
> {code}
> [KRaftMigrationDriver id=9990] Made the following ZK writes when reconciling 
> with KRaft state: {DeleteBrokerConfig=7, DeleteTopic=41, UpdateTopicConfig=41}
> {code}
> If affected by this, a quick workaround is to cause the controller to 
> failover.



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


[jira] [Created] (KAFKA-15825) KRaft controller writes empty state to ZK after migration

2023-11-14 Thread David Arthur (Jira)
David Arthur created KAFKA-15825:


 Summary: KRaft controller writes empty state to ZK after migration
 Key: KAFKA-15825
 URL: https://issues.apache.org/jira/browse/KAFKA-15825
 Project: Kafka
  Issue Type: Bug
  Components: controller
Affects Versions: 3.6.0
Reporter: David Arthur
Assignee: David Arthur
 Fix For: 3.7.0, 3.6.1


Immediately following the ZK migration, there is a race condition where the 
KRaftMigrationDriver can use an empty MetadataImage when performing the full 
"SYNC_KRAFT_TO_ZK" reconciliation. 

After the next controller failover, or when the controller loads a metadata 
snapshot, the correct state will be written to ZK. 

The symptom of this bug is that we see the migration complete, and then all the 
metadata removed from ZK. For example, 

{code}
[KRaftMigrationDriver id=9990] Completed migration of metadata from ZooKeeper 
to KRaft. 573 records were generated in 2204 ms across 51 batches. The record 
types were {TOPIC_RECORD=41, PARTITION_RECORD=410, CONFIG_RECORD=121, 
PRODUCER_IDS_RECORD=1}. The current metadata offset is now 503794 with an epoch 
of 21. Saw 6 brokers in the migrated metadata [0, 1, 2, 3, 4, 5].
{code}

immediately followed by:

{code}
[KRaftMigrationDriver id=9990] Made the following ZK writes when reconciling 
with KRaft state: {DeleteBrokerConfig=7, DeleteTopic=41, UpdateTopicConfig=41}
{code}

If affected by this, a quick workaround is to cause the controller to failover.



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


[jira] [Resolved] (KAFKA-15605) Topics marked for deletion in ZK are incorrectly migrated to KRaft

2023-11-14 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-15605.
--
Fix Version/s: 3.7.0
   Resolution: Fixed

> Topics marked for deletion in ZK are incorrectly migrated to KRaft
> --
>
> Key: KAFKA-15605
> URL: https://issues.apache.org/jira/browse/KAFKA-15605
> Project: Kafka
>  Issue Type: Bug
>  Components: controller, kraft
>Affects Versions: 3.6.0
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Major
> Fix For: 3.7.0, 3.6.1
>
>
> When migrating topics from ZooKeeper, the KRaft controller reads all the 
> topic and partition metadata from ZK directly. This includes topics which 
> have been marked for deletion by the ZK controller. After being migrated to 
> KRaft, the pending topic deletions are never completed, so it is as if the 
> delete topic request never happened.
> Since the client request to delete these topics has already been returned as 
> successful, it would be confusing to the client that the topic still existed. 
> An operator or application would need to issue another topic deletion to 
> remove these topics once the controller had moved to KRaft. If they tried to 
> create a new topic with the same name, they would receive a 
> TOPIC_ALREADY_EXISTS error.
> The migration logic should carry over pending topic deletions and resolve 
> them either as part of the migration or shortly after.
> *Note to operators:*
> To determine if a migration was affected by this, an operator can check the 
> contents of {{/admin/delete_topics}} after the KRaft controller has migrated 
> the metadata. If any topics are listed under this ZNode, they were not 
> deleted and will still be present in KRaft. At this point the operator can 
> make a determination if the topics should be re-deleted (using 
> "kafka-topics.sh --delete") or left in place. In either case, the topics 
> should be removed from {{/admin/delete_topics}} to prevent unexpected topic 
> deletion in the event of a fallback to ZK.



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


[jira] [Created] (KAFKA-15799) ZK brokers incorrectly handle KRaft metadata snapshots

2023-11-08 Thread David Arthur (Jira)
David Arthur created KAFKA-15799:


 Summary: ZK brokers incorrectly handle KRaft metadata snapshots
 Key: KAFKA-15799
 URL: https://issues.apache.org/jira/browse/KAFKA-15799
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur
Assignee: David Arthur
 Fix For: 3.6.1


While working on the fix for KAFKA-15605, I noticed that ZK brokers are 
unconditionally merging data from UpdateMetadataRequest with their existing 
MetadataCache. This is not the correct behavior when handling a metadata 
snapshot from the KRaft controller. 

For example, if a topic was deleted in KRaft and not transmitted as part of a 
delta update (e.g., during a failover) then the ZK brokers will never remove 
the topic from their cache (until they restart and rebuild their cache).



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


[jira] [Created] (KAFKA-15698) KRaft mode brokers should clean up stray partitions from migration

2023-10-26 Thread David Arthur (Jira)
David Arthur created KAFKA-15698:


 Summary: KRaft mode brokers should clean up stray partitions from 
migration
 Key: KAFKA-15698
 URL: https://issues.apache.org/jira/browse/KAFKA-15698
 Project: Kafka
  Issue Type: Improvement
Reporter: David Arthur


Follow up to KAFKA-15605. After the brokers are migrated to KRaft and the 
migration is completed, we should let the brokers clean up any partitions that 
we marked as "stray" during the migration. This would be any partition that was 
being deleted when the migration began, or any partition that was deleted, but 
not seen as deleted by StopReplica (e.g., broker down).



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


[jira] [Created] (KAFKA-15648) QuorumControllerTest#testBootstrapZkMigrationRecord is flaky

2023-10-19 Thread David Arthur (Jira)
David Arthur created KAFKA-15648:


 Summary: QuorumControllerTest#testBootstrapZkMigrationRecord is 
flaky
 Key: KAFKA-15648
 URL: https://issues.apache.org/jira/browse/KAFKA-15648
 Project: Kafka
  Issue Type: Bug
  Components: controller, unit tests
Reporter: David Arthur


Noticed that this test failed on Jenkins with 

{code}
org.apache.kafka.server.fault.FaultHandlerException: fatalFaultHandler: 
exception while completing controller activation: Should not have ZK migrations 
enabled on a cluster running metadata.version 3.0-IV1
at 
app//org.apache.kafka.controller.ActivationRecordsGenerator.recordsForNonEmptyLog(ActivationRecordsGenerator.java:154)
at 
app//org.apache.kafka.controller.ActivationRecordsGenerator.generate(ActivationRecordsGenerator.java:229)
at 
app//org.apache.kafka.controller.QuorumController$CompleteActivationEvent.generateRecordsAndResult(QuorumController.java:1237)
at 
app//org.apache.kafka.controller.QuorumController$ControllerWriteEvent.run(QuorumController.java:784)
at 
app//org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:127)
at 
app//org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:210)
at 
app//org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:181)
at java.base@11.0.16.1/java.lang.Thread.run(Thread.java:829)
Caused by: java.lang.RuntimeException: Should not have ZK migrations enabled on 
a cluster running metadata.version 3.0-IV1
... 8 more
{code}

When trying to reproduce this failure locally, I ran into a separate flaky 
failure

{code}
[2023-10-19 13:42:09,442] INFO Elected new leader: 
LeaderAndEpoch(leaderId=OptionalInt[0], epoch=1). 
(org.apache.kafka.metalog.LocalLogManager$SharedLogData:300)
[2023-10-19 13:42:09,442] DEBUG 
append(batch=LeaderChangeBatch(newLeader=LeaderAndEpoch(leaderId=OptionalInt[0],
 epoch=1)), nextEndOffset=0) 
(org.apache.kafka.metalog.LocalLogManager$SharedLogData:276)
[2023-10-19 13:42:09,442] DEBUG [LocalLogManager 0] Node 0: running log check. 
(org.apache.kafka.metalog.LocalLogManager:536)
[2023-10-19 13:42:09,442] DEBUG [LocalLogManager 0] initialized local log 
manager for node 0 (org.apache.kafka.metalog.LocalLogManager:685)
[2023-10-19 13:42:09,442] DEBUG [QuorumController id=0] Creating in-memory 
snapshot -1 (org.apache.kafka.timeline.SnapshotRegistry:203)
[2023-10-19 13:42:09,442] INFO [QuorumController id=0] Creating new 
QuorumController with clusterId K8TDRiYZQuepVQHPgwP91A. ZK migration mode is 
enabled. (org.apache.kafka.controller.QuorumController:1912)
[2023-10-19 13:42:09,442] INFO [LocalLogManager 0] Node 0: registered 
MetaLogListener 1238203422 (org.apache.kafka.metalog.LocalLogManager:703)
[2023-10-19 13:42:09,443] DEBUG [LocalLogManager 0] Node 0: running log check. 
(org.apache.kafka.metalog.LocalLogManager:536)
[2023-10-19 13:42:09,443] DEBUG [LocalLogManager 0] Node 0: Executing 
handleLeaderChange LeaderAndEpoch(leaderId=OptionalInt[0], epoch=1) 
(org.apache.kafka.metalog.LocalLogManager:578)
[2023-10-19 13:42:09,443] DEBUG [QuorumController id=0] Executing 
handleLeaderChange[1]. (org.apache.kafka.controller.QuorumController:577)
[2023-10-19 13:42:09,443] INFO [QuorumController id=0] In the new epoch 1, the 
leader is (none). (org.apache.kafka.controller.QuorumController:1179)
[2023-10-19 13:42:09,443] DEBUG [QuorumController id=0] Processed 
handleLeaderChange[1] in 25 us 
(org.apache.kafka.controller.QuorumController:510)
[2023-10-19 13:42:09,443] DEBUG [QuorumController id=0] Executing 
handleLeaderChange[1]. (org.apache.kafka.controller.QuorumController:577)
[2023-10-19 13:42:09,443] INFO [QuorumController id=0] Becoming the active 
controller at epoch 1, next write offset 1. 
(org.apache.kafka.controller.QuorumController:1175)
[2023-10-19 13:42:09,443] DEBUG [QuorumController id=0] Processed 
handleLeaderChange[1] in 34 us 
(org.apache.kafka.controller.QuorumController:510)
[2023-10-19 13:42:09,443] WARN [QuorumController id=0] Performing controller 
activation. The metadata log appears to be empty. Appending 1 bootstrap 
record(s) at metadata.version 3.4-IV0 from bootstrap source 'test'. Putting the 
controller into pre-migration mode. No metadata updates will be allowed until 
the ZK metadata has been migrated. 
(org.apache.kafka.controller.QuorumController:108)
[2023-10-19 13:42:09,443] INFO [QuorumController id=0] Replayed a 
FeatureLevelRecord setting metadata version to 3.4-IV0 
(org.apache.kafka.controller.FeatureControlManager:400)
[2023-10-19 13:42:09,443] INFO [QuorumController id=0] Replayed a 
ZkMigrationStateRecord changing the migration state from NONE to PRE_MIGRATION. 
(org.apache.kafka.controller.FeatureControlManager:421)
[2023-10-19 13:42:09,443] DEBUG append(batch=LocalRecordBatch(leaderEpoch=1, 

[jira] [Created] (KAFKA-15605) Topic marked for deletion are incorrectly migrated to KRaft

2023-10-13 Thread David Arthur (Jira)
David Arthur created KAFKA-15605:


 Summary: Topic marked for deletion are incorrectly migrated to 
KRaft
 Key: KAFKA-15605
 URL: https://issues.apache.org/jira/browse/KAFKA-15605
 Project: Kafka
  Issue Type: Bug
  Components: controller, kraft
Affects Versions: 3.6.0
Reporter: David Arthur
 Fix For: 3.6.1


When migrating topics from ZooKeeper, the KRaft controller reads all the topic 
and partition metadata from ZK directly. This includes topics which have been 
marked for deletion by the ZK controller. 

Since the client request to delete these topics has already been returned as 
successful, it would be confusing to the client that the topic still existed. 
An operator or application would need to issue another topic deletion to remove 
these topics once the controller had moved to KRaft. If they tried to create a 
new topic with the same name, they would receive a TOPIC_ALREADY_EXISTS error.

The migration logic should carry over pending topic deletions and resolve them 
either as part of the migration or shortly after.



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


[jira] [Created] (KAFKA-15552) Duplicate Producer ID blocks during ZK migration

2023-10-05 Thread David Arthur (Jira)
David Arthur created KAFKA-15552:


 Summary: Duplicate Producer ID blocks during ZK migration
 Key: KAFKA-15552
 URL: https://issues.apache.org/jira/browse/KAFKA-15552
 Project: Kafka
  Issue Type: Bug
Affects Versions: 3.5.1, 3.4.1, 3.5.0, 3.4.0, 3.6.0
Reporter: David Arthur
Assignee: David Arthur
 Fix For: 3.4.2, 3.5.2, 3.6.1


When migrating producer ID blocks from ZK to KRaft, we are taking the current 
producer ID block from ZK and writing it's "firstProducerId" into the producer 
IDs KRaft record. However, in KRaft we store the _next_ producer ID block in 
the log rather than storing the current block like ZK does. The end result is 
that the first block given to a caller of AllocateProducerIds is a duplicate of 
the last block allocated in ZK mode.

 

This can result in duplicate producer IDs being given to transactional or 
idempotent producers. In the case of transactional producers, this can cause 
long term problems since the producer IDs are persisted and reused for a long 
time.


The time between the last producer ID block being allocated by the ZK 
controller and all the brokers being restarted following the metadata migration 
is when this bug is possible.
 

Symptoms of this bug will include ReplicaManager OutOfOrderSequenceException 
and possibly some producer epoch validation errors. To see if a cluster is 
affected by this bug, search for the offending producer ID and see if it is 
being used by more than one producer.

 

For example, the following error was observed
{code}
Out of order sequence number for producer 376000 at offset 381338 in partition 
REDACTED: 0 (incoming seq. number), 21 (current end sequence number) 
{code}

Then searching for "376000" on 
org.apache.kafka.clients.producer.internals.TransactionManager logs, two 
brokers both show the same producer ID being provisioned

{code}
Broker 0 [Producer clientId=REDACTED-0] ProducerId set to 376000 with epoch 1
Broker 5 [Producer clientId=REDACTED-1] ProducerId set to 376000 with epoch 1
{code}





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


[jira] [Created] (KAFKA-15532) ZkWriteBehindLag should not be reported by inactive controllers

2023-10-03 Thread David Arthur (Jira)
David Arthur created KAFKA-15532:


 Summary: ZkWriteBehindLag should not be reported by inactive 
controllers
 Key: KAFKA-15532
 URL: https://issues.apache.org/jira/browse/KAFKA-15532
 Project: Kafka
  Issue Type: Bug
Affects Versions: 3.6.0
Reporter: David Arthur


Since only the active controller is performing the dual-write to ZK during a 
migration, it should be the only controller to report the ZkWriteBehindLag 
metric. 

 

Currently, if the controller fails over during a migration, the previous active 
controller will incorrectly report its last value for ZkWriteBehindLag forever. 
Instead, it should report zero.



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


[jira] [Resolved] (KAFKA-15450) Disable ZK migration when JBOD configured

2023-09-12 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-15450.
--
Resolution: Fixed

> Disable ZK migration when JBOD configured
> -
>
> Key: KAFKA-15450
> URL: https://issues.apache.org/jira/browse/KAFKA-15450
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.4.1, 3.6.0, 3.5.1
>Reporter: David Arthur
>Priority: Critical
> Fix For: 3.6.0, 3.4.2, 3.5.2
>
>
> Since JBOD is not yet supported in KRaft (see 
> [KIP-858|https://cwiki.apache.org/confluence/display/KAFKA/KIP-858%3A+Handle+JBOD+broker+disk+failure+in+KRaft]),
>  we need to prevent users from starting a ZK to KRaft migration if JBOD is 
> used.
>  



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


[jira] [Created] (KAFKA-15450) Disable ZK migration when JBOD configured

2023-09-11 Thread David Arthur (Jira)
David Arthur created KAFKA-15450:


 Summary: Disable ZK migration when JBOD configured
 Key: KAFKA-15450
 URL: https://issues.apache.org/jira/browse/KAFKA-15450
 Project: Kafka
  Issue Type: Bug
Affects Versions: 3.5.1, 3.4.1, 3.6.0
Reporter: David Arthur
 Fix For: 3.6.0


Since JBOD is not yet supported in KRaft (see 
[KIP-858|[https://cwiki.apache.org/confluence/display/KAFKA/KIP-858%3A+Handle+JBOD+broker+disk+failure+in+KRaft]]),
 we need to prevent users from starting a ZK to KRaft migration if JBOD is used.

 



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


[jira] [Resolved] (KAFKA-15441) Broker sessions can time out during ZK migration

2023-09-08 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-15441.
--
Resolution: Fixed

> Broker sessions can time out during ZK migration
> 
>
> Key: KAFKA-15441
> URL: https://issues.apache.org/jira/browse/KAFKA-15441
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.6.0
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Blocker
> Fix For: 3.6.0
>
>
> When a ZK to KRaft migration takes more than a few seconds to complete, the 
> sessions between the ZK brokers and the KRaft controller will expire. This 
> appears to be due to the heartbeat events being blocked in the purgatory on 
> the controller.
> The side effect of this expiration is that after the metadata is migrated, 
> the KRaft controller will immediately fence all of the brokers and remove 
> them from ISRs. This leads to a mass leadership change that can cause large 
> latency spikes on the brokers.



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


[jira] [Resolved] (KAFKA-15435) KRaft migration record counts in log message are incorrect

2023-09-08 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-15435.
--
Resolution: Fixed

> KRaft migration record counts in log message are incorrect
> --
>
> Key: KAFKA-15435
> URL: https://issues.apache.org/jira/browse/KAFKA-15435
> Project: Kafka
>  Issue Type: Bug
>  Components: kraft
>Affects Versions: 3.6.0
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Blocker
> Fix For: 3.6.0
>
>
> The counting logic in MigrationManifest is incorrect and produces invalid 
> output. This information is critical for users wanting to validate the result 
> of a migration.
>  
> {code}
> Completed migration of metadata from ZooKeeper to KRaft. 7117 records were 
> generated in 54253 ms across 1629 batches. The record types were 
> {TOPIC_RECORD=2, CONFIG_RECORD=2, PARTITION_RECORD=2, 
> ACCESS_CONTROL_ENTRY_RECORD=2, PRODUCER_IDS_RECORD=1}. 
> {code}
> Due to the logic bug, the counts will never exceed 2.



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


[jira] [Created] (KAFKA-15441) Broker sessions can time out during ZK migration

2023-09-06 Thread David Arthur (Jira)
David Arthur created KAFKA-15441:


 Summary: Broker sessions can time out during ZK migration
 Key: KAFKA-15441
 URL: https://issues.apache.org/jira/browse/KAFKA-15441
 Project: Kafka
  Issue Type: Bug
Affects Versions: 3.6.0
Reporter: David Arthur
Assignee: David Arthur


When a ZK to KRaft migration takes more than a few seconds to complete, the 
sessions between the ZK brokers and the KRaft controller will expire. This 
appears to be due to the heartbeat events being blocked in the purgatory on the 
controller.

The side effect of this expiration is that after the metadata is migrated, the 
KRaft controller will immediately fence all of the brokers and remove them from 
ISRs. This leads to a mass leadership change that can cause large latency 
spikes on the brokers.



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


[jira] [Created] (KAFKA-15435) KRaft migration record counts in log message are incorrect

2023-09-05 Thread David Arthur (Jira)
David Arthur created KAFKA-15435:


 Summary: KRaft migration record counts in log message are incorrect
 Key: KAFKA-15435
 URL: https://issues.apache.org/jira/browse/KAFKA-15435
 Project: Kafka
  Issue Type: Bug
  Components: kraft
Affects Versions: 3.6.0
Reporter: David Arthur


The counting logic in MigrationManifest is incorrect and produces invalid 
output. This information is critical for users wanting to validate the result 
of a migration.

 
{code}
Completed migration of metadata from ZooKeeper to KRaft. 7117 records were 
generated in 54253 ms across 1629 batches. The record types were 
{TOPIC_RECORD=2, CONFIG_RECORD=2, PARTITION_RECORD=2, 
ACCESS_CONTROL_ENTRY_RECORD=2, PRODUCER_IDS_RECORD=1}. 
{code}

Due to the logic bug, the counts will never exceed 2.



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


[jira] [Created] (KAFKA-15389) MetadataLoader may publish an empty image on first start

2023-08-21 Thread David Arthur (Jira)
David Arthur created KAFKA-15389:


 Summary: MetadataLoader may publish an empty image on first start
 Key: KAFKA-15389
 URL: https://issues.apache.org/jira/browse/KAFKA-15389
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur


When first loading from an empty log, there is a case where MetadataLoader can 
publish an image before the bootstrap records are processed. This isn't exactly 
incorrect, since all components implicitly start from the empty image state, 
but it might be unexpected for some MetadataPublishers. 

 

For example, in KRaftMigrationDriver, if an old MetadataVersion is encountered, 
the driver transitions to the INACTIVE state.



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


[jira] [Created] (KAFKA-15381) Controller waiting for migration should only allow failover when transactions are supported

2023-08-18 Thread David Arthur (Jira)
David Arthur created KAFKA-15381:


 Summary: Controller waiting for migration should only allow 
failover when transactions are supported
 Key: KAFKA-15381
 URL: https://issues.apache.org/jira/browse/KAFKA-15381
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur
 Fix For: 3.6.0


After a KRaft controller starts up in migration mode, it enters the 
"pre-migration" state. Unless transactions are supported, it is not safe for 
the controller to fail over in pre-migration mode. This is because a migration 
could have been partially committed when the failover occurs. 



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


[jira] [Created] (KAFKA-15374) ZK migration fails on configs for default broker resource

2023-08-17 Thread David Arthur (Jira)
David Arthur created KAFKA-15374:


 Summary: ZK migration fails on configs for default broker resource
 Key: KAFKA-15374
 URL: https://issues.apache.org/jira/browse/KAFKA-15374
 Project: Kafka
  Issue Type: Bug
Affects Versions: 3.5.1, 3.4.1
Reporter: David Arthur
 Fix For: 3.6.0, 3.4.2, 3.5.2


This error was seen while performing a ZK to KRaft migration on a cluster with 
configs for the default broker resource

 
{code:java}
java.lang.NumberFormatException: For input string: ""
at 
java.base/java.lang.NumberFormatException.forInputString(NumberFormatException.java:67)
at java.base/java.lang.Integer.parseInt(Integer.java:678)
at java.base/java.lang.Integer.valueOf(Integer.java:999)
at 
kafka.zk.ZkMigrationClient.$anonfun$migrateBrokerConfigs$2(ZkMigrationClient.scala:371)
at 
kafka.zk.migration.ZkConfigMigrationClient.$anonfun$iterateBrokerConfigs$1(ZkConfigMigrationClient.scala:174)
at 
kafka.zk.migration.ZkConfigMigrationClient.$anonfun$iterateBrokerConfigs$1$adapted(ZkConfigMigrationClient.scala:156)
at 
scala.collection.immutable.BitmapIndexedMapNode.foreach(HashMap.scala:1076)
at scala.collection.immutable.HashMap.foreach(HashMap.scala:1083)
at 
kafka.zk.migration.ZkConfigMigrationClient.iterateBrokerConfigs(ZkConfigMigrationClient.scala:156)
at 
kafka.zk.ZkMigrationClient.migrateBrokerConfigs(ZkMigrationClient.scala:370)
at 
kafka.zk.ZkMigrationClient.cleanAndMigrateAllMetadata(ZkMigrationClient.scala:530)
at 
org.apache.kafka.metadata.migration.KRaftMigrationDriver$MigrateMetadataEvent.run(KRaftMigrationDriver.java:618)
at 
org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:127)
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:833)
at org.apache.kafka.common.utils.KafkaThread.run(KafkaThread.java:64) 
{code}
 

This is due to not considering the default resource type when we collect the 
broker IDs in ZkMigrationClient#migrateBrokerConfigs.

 

 



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


[jira] [Resolved] (KAFKA-15263) KRaftMigrationDriver can run the migration twice

2023-07-28 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-15263.
--
Resolution: Fixed

> KRaftMigrationDriver can run the migration twice
> 
>
> Key: KAFKA-15263
> URL: https://issues.apache.org/jira/browse/KAFKA-15263
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.5.0, 3.5.1
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Blocker
> Fix For: 3.6.0, 3.5.2
>
>
> There is a narrow race condition in KRaftMigrationDriver where a PollEvent 
> can run that sees the internal state as ZK_MIGRATION and is immediately 
> followed by another poll event (due to external call to {{{}wakeup(){}}}) 
> that results in two MigrateMetadataEvent being enqueued. 
> Since MigrateMetadataEvent lacks a check on the internal state, this causes 
> the metadata migration to occur twice. 



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


[jira] [Created] (KAFKA-15263) KRaftMigrationDriver can run the migration twice

2023-07-27 Thread David Arthur (Jira)
David Arthur created KAFKA-15263:


 Summary: KRaftMigrationDriver can run the migration twice
 Key: KAFKA-15263
 URL: https://issues.apache.org/jira/browse/KAFKA-15263
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur
Assignee: David Arthur


There is a narrow race condition in KRaftMigrationDriver where a PollEvent can 
run that sees the internal state as ZK_MIGRATION and is immediately followed by 
another poll event (due to external call to {{{}wakeup(){}}}) that results in 
two MigrateMetadataEvent being enqueued. 

Since MigrateMetadataEvent lacks a check on the internal state, this causes the 
metadata migration to occur twice. 



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


[jira] [Created] (KAFKA-15230) ApiVersions data between controllers is not reliable

2023-07-21 Thread David Arthur (Jira)
David Arthur created KAFKA-15230:


 Summary: ApiVersions data between controllers is not reliable
 Key: KAFKA-15230
 URL: https://issues.apache.org/jira/browse/KAFKA-15230
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur


While testing ZK migrations, I noticed a case where the controller was not 
starting the migration due to the missing ApiVersions data from other 
controllers. This was unexpected because the quorum was running and the 
followers were replicating the metadata log as expected. After examining a heap 
dump of the leader, it was in fact the case that the ApiVersions map of 
NodeApiVersions was empty.

 

After further investigation and offline discussion with [~jsancio], we realized 
that after the initial leader election, the connection from the Raft leader to 
the followers will become idle and eventually timeout and close. This causes 
NetworkClient to purge the NodeApiVersions data for the closed connections.

 

There are two main side effects of this behavior: 

1) If migrations are not started within the idle timeout period (10 minutes, by 
default), then they will not be able to be started. After this timeout period, 
I was unable to restart the controllers in such a way that the leader had 
active connections with all followers.

2) Dynamically updating features, such as "metadata.version", is not guaranteed 
to be safe

 

There is a partial workaround for the migration issue. If we set "
connections.max.idle.ms" to -1, the Raft leader will never disconnect from the 
followers. However, if a follower restarts, the leader will not re-establish a 
connection.
 
The feature update issue has no safe workarounds.



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


[jira] [Created] (KAFKA-15196) Additional ZK migration metrics

2023-07-17 Thread David Arthur (Jira)
David Arthur created KAFKA-15196:


 Summary: Additional ZK migration metrics
 Key: KAFKA-15196
 URL: https://issues.apache.org/jira/browse/KAFKA-15196
 Project: Kafka
  Issue Type: Sub-task
Reporter: David Arthur
Assignee: David Arthur


This issue is to track the remaining metrics defined in KIP-866. So far, we 
have ZkMigrationState and ZkWriteBehindLag.



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


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

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


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


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



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


[jira] [Resolved] (KAFKA-15098) KRaft migration does not proceed and broker dies if authorizer.class.name is set

2023-06-22 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-15098.
--
Resolution: Fixed

> KRaft migration does not proceed and broker dies if authorizer.class.name is 
> set
> 
>
> Key: KAFKA-15098
> URL: https://issues.apache.org/jira/browse/KAFKA-15098
> Project: Kafka
>  Issue Type: Bug
>  Components: kraft
>Affects Versions: 3.5.0
>Reporter: Ron Dagostino
>Assignee: David Arthur
>Priority: Blocker
> Fix For: 3.6.0, 3.5.1
>
>
> [ERROR] 2023-06-16 20:14:14,298 [main] kafka.Kafka$ - Exiting Kafka due to 
> fatal exception
> java.lang.IllegalArgumentException: requirement failed: ZooKeeper migration 
> does not yet support authorizers. Remove authorizer.class.name before 
> performing a migration.



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


[jira] [Resolved] (KAFKA-15109) ISR shrink/expand issues on ZK brokers during migration

2023-06-22 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-15109.
--
Resolution: Fixed

> ISR shrink/expand issues on ZK brokers during migration
> ---
>
> Key: KAFKA-15109
> URL: https://issues.apache.org/jira/browse/KAFKA-15109
> Project: Kafka
>  Issue Type: Bug
>  Components: kraft, replication
>Affects Versions: 3.6.0
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Critical
> Fix For: 3.6.0
>
>
> KAFKA-15021 introduced a new controller behavior that avoids increasing the 
> leader epoch during the controlled shutdown scenario. This prevents some 
> unnecessary thrashing of metadata and threads on the brokers and clients. 
> While a cluster is in a KIP-866 migration and has a KRaft controller with ZK 
> brokers, we cannot employ this leader epoch bump avoidance. The ZK brokers 
> must have the leader epoch bump in order for ReplicaManager to react to the 
> LeaderAndIsrRequest.



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


[jira] [Created] (KAFKA-15109) ISR not expanding on ZK brokers during migration

2023-06-20 Thread David Arthur (Jira)
David Arthur created KAFKA-15109:


 Summary: ISR not expanding on ZK brokers during migration
 Key: KAFKA-15109
 URL: https://issues.apache.org/jira/browse/KAFKA-15109
 Project: Kafka
  Issue Type: Bug
  Components: kraft, replication
Affects Versions: 3.5.0
Reporter: David Arthur


KAFKA-15021 introduced a new controller behavior that avoids increasing the 
leader epoch during the controlled shutdown scenario. This prevents some 
unnecessary thrashing of metadata and threads on the brokers and clients. 

While a cluster is in a KIP-866 migration and has a KRaft controller with ZK 
brokers, we cannot employ this leader epoch bump avoidance. The ZK brokers must 
have the leader epoch bump in order for the ISR expansion to complete.



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


[jira] [Created] (KAFKA-15073) Automation for old/inactive PRs

2023-06-07 Thread David Arthur (Jira)
David Arthur created KAFKA-15073:


 Summary: Automation for old/inactive PRs
 Key: KAFKA-15073
 URL: https://issues.apache.org/jira/browse/KAFKA-15073
 Project: Kafka
  Issue Type: Improvement
  Components: build
Reporter: David Arthur


Following from a discussion on the mailing list. It would be nice to 
automatically triage inactive PRs. There are currently over 1000 open PRs. Most 
likely a majority of these will not ever be merged and should be closed.



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


[jira] [Created] (KAFKA-15017) New ClientQuotas are not written to ZK from snapshot

2023-05-23 Thread David Arthur (Jira)
David Arthur created KAFKA-15017:


 Summary: New ClientQuotas are not written to ZK from snapshot 
 Key: KAFKA-15017
 URL: https://issues.apache.org/jira/browse/KAFKA-15017
 Project: Kafka
  Issue Type: Bug
  Components: kraft
Affects Versions: 3.5.0
Reporter: David Arthur


Similar issue to KAFKA-15009



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


[jira] [Resolved] (KAFKA-14805) KRaft Controller shouldn't allow metadata updates before migration starts

2023-05-06 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14805.
--
Resolution: Fixed

> KRaft Controller shouldn't allow metadata updates before migration starts
> -
>
> Key: KAFKA-14805
> URL: https://issues.apache.org/jira/browse/KAFKA-14805
> Project: Kafka
>  Issue Type: Sub-task
>  Components: kraft
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Critical
> Fix For: 3.5.0
>
>
> When starting a ZK to KRaft migration, the new KRaft quorum should not accept 
> external metadata updates from things like CreateTopics or 
> AllocateProducerIds. Having metadata present in the log prior to the 
> migration can lead to undefined state, which is not great.
> This is currently causing test failures on trunk because some producer is 
> allocating a producer ID between the time the KRaft quorum starts and the 
> migration starts.



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


[jira] [Resolved] (KAFKA-14840) Handle KRaft snapshots in dual-write mode

2023-05-06 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14840.
--
Fix Version/s: (was: 3.4.1)
   Resolution: Fixed

> Handle KRaft snapshots in dual-write mode
> -
>
> Key: KAFKA-14840
> URL: https://issues.apache.org/jira/browse/KAFKA-14840
> Project: Kafka
>  Issue Type: Sub-task
>  Components: kraft
>Affects Versions: 3.4.0
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Blocker
> Fix For: 3.5.0
>
>
> While the KRaft controller is making writes back to ZK during the migration, 
> we need to handle the case when a snapshot is loaded. This can happen for a 
> number of reasons in KRaft.
> The difficulty here is we will need to compare the loaded snapshot with the 
> entire state in ZK. Most likely, this will be a very expensive operation.
> Without this, dual-write mode cannot safely tolerate a snapshot being loaded, 
> so marking this as a 3.5 blocker.



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


[jira] [Created] (KAFKA-14964) ClientQuotaMetadataManager should not suppress exceptions

2023-05-03 Thread David Arthur (Jira)
David Arthur created KAFKA-14964:


 Summary: ClientQuotaMetadataManager should not suppress exceptions
 Key: KAFKA-14964
 URL: https://issues.apache.org/jira/browse/KAFKA-14964
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur


As MetadataLoader calls each MetadataPublisher upon receiving new records from 
the controller, it surrounds the call with a try-catch block in order to pass 
exceptions to a FaultHandler. The FaultHandler used by MetadataLoader is 
essential for us to learn about metadata errors on the broker since it 
increments the metadata loader error JMX metric.

ClientQuotaMetadataManager is in the update path for ClientQuota metadata 
updates and is capturing exceptions. This means validation errors (like invalid 
quotas) will not be seen by the FaultHandler, and the JMX metric will not get 
incremented.



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


[jira] [Created] (KAFKA-14939) Only expose ZkMigrationState metric if metadata.version supports it

2023-04-26 Thread David Arthur (Jira)
David Arthur created KAFKA-14939:


 Summary: Only expose ZkMigrationState metric if metadata.version 
supports it
 Key: KAFKA-14939
 URL: https://issues.apache.org/jira/browse/KAFKA-14939
 Project: Kafka
  Issue Type: Sub-task
Affects Versions: 3.5.0
Reporter: David Arthur


We should only expose the KafkaController.ZkMigrationState JMX metric if the 
cluster is running on a metadata.version that supports migrations.



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


[jira] [Created] (KAFKA-14934) KafkaClusterTestKit makes FaultHandler accessible

2023-04-25 Thread David Arthur (Jira)
David Arthur created KAFKA-14934:


 Summary: KafkaClusterTestKit makes FaultHandler accessible
 Key: KAFKA-14934
 URL: https://issues.apache.org/jira/browse/KAFKA-14934
 Project: Kafka
  Issue Type: Improvement
  Components: unit tests
Reporter: David Arthur


In KafkaClusterTestKit, we use a mock fault handler to avoid exiting the 
process during tests. It would be useful to expose this fault handler so tests 
could verify certain fault conditions (like a broker/controller failing to 
start)



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


[jira] [Created] (KAFKA-14918) KRaft controller sending ZK controller RPCs to KRaft brokers

2023-04-18 Thread David Arthur (Jira)
David Arthur created KAFKA-14918:


 Summary: KRaft controller sending ZK controller RPCs to KRaft 
brokers
 Key: KAFKA-14918
 URL: https://issues.apache.org/jira/browse/KAFKA-14918
 Project: Kafka
  Issue Type: Sub-task
Reporter: David Arthur
Assignee: David Arthur


During the migration, when upgrading a ZK broker to KRaft, the controller is 
incorrectly sending UpdateMetadata requests to the KRaft controller. 



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


[jira] [Resolved] (KAFKA-14796) Migrate ZK ACLs to KRaft

2023-04-17 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14796.
--
Resolution: Fixed

Removed the 3.4.1 fix version since we're probably not back-porting this.

> Migrate ZK ACLs to KRaft
> 
>
> Key: KAFKA-14796
> URL: https://issues.apache.org/jira/browse/KAFKA-14796
> Project: Kafka
>  Issue Type: Sub-task
>  Components: kraft
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Major
> Fix For: 3.5.0
>
>




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


[jira] [Created] (KAFKA-14909) KRaft Controllers not setting ZkMigrationReady tagged field

2023-04-14 Thread David Arthur (Jira)
David Arthur created KAFKA-14909:


 Summary: KRaft Controllers not setting ZkMigrationReady tagged 
field
 Key: KAFKA-14909
 URL: https://issues.apache.org/jira/browse/KAFKA-14909
 Project: Kafka
  Issue Type: Bug
  Components: kraft
Affects Versions: 3.4.0
Reporter: David Arthur
 Fix For: 3.5.0, 3.4.1


When sending ApiVersionsResponse to other controllers, the KRaft controller is 
not setting the ZkMigrationReady field. This means, we can't determine if the 
full KRaft quorum has been properly configured for a migration before 
triggering the migration.

As a result, we could start the migration on controller A (which was properly 
configured), then fail over to controller B (which was not properly configured) 
and no longer be in dual-write mode.

The fix is to properly set the ZkMigrationReady tagged field, and to make use 
of it in KRaftMigrationDriver



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


[jira] [Created] (KAFKA-14840) Handle KRaft snapshots in dual-write mode

2023-03-23 Thread David Arthur (Jira)
David Arthur created KAFKA-14840:


 Summary: Handle KRaft snapshots in dual-write mode
 Key: KAFKA-14840
 URL: https://issues.apache.org/jira/browse/KAFKA-14840
 Project: Kafka
  Issue Type: Sub-task
  Components: kraft
Affects Versions: 3.4.0
Reporter: David Arthur
Assignee: David Arthur
 Fix For: 3.5.0, 3.4.1


While the KRaft controller is making writes back to ZK during the migration, we 
need to handle the case when a snapshot is loaded. This can happen for a number 
of reasons in KRaft.

The difficulty here is we will need to compare the loaded snapshot with the 
entire state in ZK. Most likely, this will be a very expensive operation.

Without this, dual-write mode cannot safely tolerate a snapshot being loaded, 
so marking this as a 3.5 blocker.



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


[jira] [Created] (KAFKA-14833) ZK migration cannot start if a ZK controller was not previously elected

2023-03-22 Thread David Arthur (Jira)
David Arthur created KAFKA-14833:


 Summary: ZK migration cannot start if a ZK controller was not 
previously elected
 Key: KAFKA-14833
 URL: https://issues.apache.org/jira/browse/KAFKA-14833
 Project: Kafka
  Issue Type: Bug
  Components: kraft
Affects Versions: 3.4.0
Reporter: David Arthur


During the BecomeZkController event in KRaftMigrationDriver, an error occurs if 
there was not data in the /controller_epoch znode.

{code}
[2023-03-22 14:33:06,082] ERROR Had an exception in BecomeZkControllerEvent 
(org.apache.kafka.metadata.migration.KRaftMigrationDriver)
java.lang.IllegalStateException: Cannot register KRaft controller 3001 as the 
active controller since there is no ZK controller epoch present.
at 
kafka.zk.KafkaZkClient.tryRegisterKRaftControllerAsActiveController(KafkaZkClient.scala:201)
at 
kafka.zk.ZkMigrationClient.claimControllerLeadership(ZkMigrationClient.scala:60)
at 
org.apache.kafka.metadata.migration.KRaftMigrationDriver.apply(KRaftMigrationDriver.java:143)
at 
org.apache.kafka.metadata.migration.KRaftMigrationDriver.access$800(KRaftMigrationDriver.java:57)
at 
org.apache.kafka.metadata.migration.KRaftMigrationDriver$BecomeZkControllerEvent.run(KRaftMigrationDriver.java:357)
at 
org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:121)
at 
org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:201)
at 
org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:175)
at java.lang.Thread.run(Thread.java:750)
{code}

This mainly affects tests when we start a KRaft quorum first, and then the 
(new) ZK cluster. The migration driver will kick off when the ZK brokers 
register but it will race with KafkaController on the /controller_epoch 
creation.



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


[jira] [Created] (KAFKA-14805) KRaft Controller doesn't allow metadata updates before migration starts

2023-03-13 Thread David Arthur (Jira)
David Arthur created KAFKA-14805:


 Summary: KRaft Controller doesn't allow metadata updates before 
migration starts
 Key: KAFKA-14805
 URL: https://issues.apache.org/jira/browse/KAFKA-14805
 Project: Kafka
  Issue Type: Sub-task
  Components: kraft
Reporter: David Arthur
 Fix For: 3.5.0, 3.4.1


When starting a ZK to KRaft migration, the new KRaft quorum should not accept 
external metadata updates from things like CreateTopics or AllocateProducerIds. 
Having metadata present in the log prior to the migration can lead to undefined 
state, which is not great.

This is currently causing test failures on trunk because some producer is 
allocating a producer ID between the time the KRaft quorum starts and the 
migration starts.



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


[jira] [Created] (KAFKA-14804) Connect docs fail to build with Gradle Swagger plugin 2.2.8

2023-03-13 Thread David Arthur (Jira)
David Arthur created KAFKA-14804:


 Summary: Connect docs fail to build with Gradle Swagger plugin 
2.2.8
 Key: KAFKA-14804
 URL: https://issues.apache.org/jira/browse/KAFKA-14804
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur


There is an incompatibility somewhere between versions 2.2.0 and 2.2.8 that 
cause the following error when building the connect docs:

{code}
Caused by: org.gradle.api.GradleException: 
io.swagger.v3.jaxrs2.integration.SwaggerLoader.setOpenAPI31(java.lang.Boolean)
at 
io.swagger.v3.plugins.gradle.tasks.ResolveTask.resolve(ResolveTask.java:458)
at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at org.gradle.internal.reflect.JavaMethod.invoke(JavaMethod.java:125)
at 
org.gradle.api.internal.project.taskfactory.StandardTaskAction.doExecute(StandardTaskAction.java:58)
at 
org.gradle.api.internal.project.taskfactory.StandardTaskAction.execute(StandardTaskAction.java:51)
at 
org.gradle.api.internal.project.taskfactory.StandardTaskAction.execute(StandardTaskAction.java:29)
at 
org.gradle.api.internal.tasks.execution.TaskExecution$3.run(TaskExecution.java:242)
at 
org.gradle.internal.operations.DefaultBuildOperationRunner$1.execute(DefaultBuildOperationRunner.java:29)
at 
org.gradle.internal.operations.DefaultBuildOperationRunner$1.execute(DefaultBuildOperationRunner.java:26)
at 
org.gradle.internal.operations.DefaultBuildOperationRunner$2.execute(DefaultBuildOperationRunner.java:66)
at 
org.gradle.internal.operations.DefaultBuildOperationRunner$2.execute(DefaultBuildOperationRunner.java:59)
at 
org.gradle.internal.operations.DefaultBuildOperationRunner.execute(DefaultBuildOperationRunner.java:157)
at 
org.gradle.internal.operations.DefaultBuildOperationRunner.execute(DefaultBuildOperationRunner.java:59)
at 
org.gradle.internal.operations.DefaultBuildOperationRunner.run(DefaultBuildOperationRunner.java:47)
at 
org.gradle.internal.operations.DefaultBuildOperationExecutor.run(DefaultBuildOperationExecutor.java:68)
at 
org.gradle.api.internal.tasks.execution.TaskExecution.executeAction(TaskExecution.java:227)
at 
org.gradle.api.internal.tasks.execution.TaskExecution.executeActions(TaskExecution.java:210)
at 
org.gradle.api.internal.tasks.execution.TaskExecution.executeWithPreviousOutputFiles(TaskExecution.java:193)
at 
org.gradle.api.internal.tasks.execution.TaskExecution.execute(TaskExecution.java:166)
at 
org.gradle.internal.execution.steps.ExecuteStep.executeInternal(ExecuteStep.java:93)
at 
org.gradle.internal.execution.steps.ExecuteStep.access$000(ExecuteStep.java:44)
at 
org.gradle.internal.execution.steps.ExecuteStep$1.call(ExecuteStep.java:57)
at 
org.gradle.internal.execution.steps.ExecuteStep$1.call(ExecuteStep.java:54)
at 
org.gradle.internal.operations.DefaultBuildOperationRunner$CallableBuildOperationWorker.execute(DefaultBuildOperationRunner.java:204)
at 
org.gradle.internal.operations.DefaultBuildOperationRunner$CallableBuildOperationWorker.execute(DefaultBuildOperationRunner.java:199)
at 
org.gradle.internal.operations.DefaultBuildOperationRunner$2.execute(DefaultBuildOperationRunner.java:66)
at 
org.gradle.internal.operations.DefaultBuildOperationRunner$2.execute(DefaultBuildOperationRunner.java:59)
at 
org.gradle.internal.operations.DefaultBuildOperationRunner.execute(DefaultBuildOperationRunner.java:157)
at 
org.gradle.internal.operations.DefaultBuildOperationRunner.execute(DefaultBuildOperationRunner.java:59)
at 
org.gradle.internal.operations.DefaultBuildOperationRunner.call(DefaultBuildOperationRunner.java:53)
at 
org.gradle.internal.operations.DefaultBuildOperationExecutor.call(DefaultBuildOperationExecutor.java:73)
at 
org.gradle.internal.execution.steps.ExecuteStep.execute(ExecuteStep.java:54)
at 
org.gradle.internal.execution.steps.ExecuteStep.execute(ExecuteStep.java:44)
at 
org.gradle.internal.execution.steps.RemovePreviousOutputsStep.execute(RemovePreviousOutputsStep.java:67)
at 
org.gradle.internal.execution.steps.RemovePreviousOutputsStep.execute(RemovePreviousOutputsStep.java:37)
at 
org.gradle.internal.execution.steps.CancelExecutionStep.execute(CancelExecutionStep.java:41)
at 
org.gradle.internal.execution.steps.TimeoutStep.executeWithoutTimeout(TimeoutStep.java:74)
at 
org.gradle.internal.execution.steps.TimeoutStep.execute(TimeoutStep.java:55)
at 

[jira] [Created] (KAFKA-14796) Migrate ZK ACLs to KRaft

2023-03-08 Thread David Arthur (Jira)
David Arthur created KAFKA-14796:


 Summary: Migrate ZK ACLs to KRaft
 Key: KAFKA-14796
 URL: https://issues.apache.org/jira/browse/KAFKA-14796
 Project: Kafka
  Issue Type: Sub-task
  Components: kraft
Reporter: David Arthur
Assignee: David Arthur
 Fix For: 3.5.0, 3.4.1






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


[jira] [Resolved] (KAFKA-14769) NPE in ControllerMetricsManager when upgrading from old KRaft version

2023-02-28 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14769.
--
Resolution: Invalid

> NPE in ControllerMetricsManager when upgrading from old KRaft version
> -
>
> Key: KAFKA-14769
> URL: https://issues.apache.org/jira/browse/KAFKA-14769
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.4.0
>Reporter: David Arthur
>Assignee: Jose Armando Garcia Sancio
>Priority: Critical
> Fix For: 3.5.0, 3.4.1
>
>
> In older KRaft versions, we could see a ConfigRecord for a topic config 
> appear before the TopicRecord in a batch.
> When upgrading from an older KRaft version (e.g., 3.1), the latest code in 
> the KRaft controller hits an NPE when it encounters a ConfigRecord for a 
> topic config before the TopicRecord. This was introduced relatively recently 
> by KAFKA-14457



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


[jira] [Created] (KAFKA-14769) NPE in ControllerMetricsManager when upgrading from old KRaft version

2023-02-28 Thread David Arthur (Jira)
David Arthur created KAFKA-14769:


 Summary: NPE in ControllerMetricsManager when upgrading from old 
KRaft version
 Key: KAFKA-14769
 URL: https://issues.apache.org/jira/browse/KAFKA-14769
 Project: Kafka
  Issue Type: Bug
Affects Versions: 3.4.0
Reporter: David Arthur
Assignee: Jose Armando Garcia Sancio
 Fix For: 3.5.0, 3.4.1


In older KRaft versions, we could see a ConfigRecord for a topic config appear 
before the TopicRecord in a batch.

When upgrading from an older KRaft version (e.g., 3.1), the latest code in the 
KRaft controller hits an NPE when it encounters a ConfigRecord for a topic 
config before the TopicRecord. This was introduced relatively recently by 
KAFKA-14457



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


[jira] [Created] (KAFKA-14712) Confusing error when writing downgraded FeatureImage

2023-02-13 Thread David Arthur (Jira)
David Arthur created KAFKA-14712:


 Summary: Confusing error when writing downgraded FeatureImage
 Key: KAFKA-14712
 URL: https://issues.apache.org/jira/browse/KAFKA-14712
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur


We have logic in ImageWriterOptions which forces any MetadataVersion lower than 
3.3-IV0 to be treated as 3.0-IV1. This was because FeatureLevel records were 
not supported before 3.3-IV0. 

When FeatureLevel is written at an older version, the "loss handler" produces 
an error message warning the user that some metadata is being lost.

For example, when writing a FeatureImage with flag "foo" at MetadataVersion 
3.2-IV0, we get a message like:

> Metadata has been lost because the following could not be represented in 
> metadata version 3.0-IV1: feature flag(s): foo

This is confusing since we told the image builder to use MetadataVersion 
3.2-IV0, but 3.0-IV1 appears in the text.



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


[jira] [Resolved] (KAFKA-14697) KRaft controller should not send LISR to KRaft brokers during migration

2023-02-09 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14697.
--
Fix Version/s: (was: 3.5.0)
   (was: 3.4.1)
   Resolution: Duplicate

> KRaft controller should not send LISR to KRaft brokers during migration
> ---
>
> Key: KAFKA-14697
> URL: https://issues.apache.org/jira/browse/KAFKA-14697
> Project: Kafka
>  Issue Type: Bug
>  Components: kraft
>Affects Versions: 3.4.0
>Reporter: David Arthur
>Priority: Major
> Attachments: test_online_migration.tar.gz
>
>
> During a migration, after a broker is restarted in KRaft mode the KRaft 
> controller is still sending it LISR as if it were a ZK broker. This results 
> in errors like:
> {code}
> [2023-02-09 14:53:26,892] ERROR Closing socket for 
> 172.19.0.4:9092-172.19.0.9:32876-0 because of error (kafka.network.Processor)
> org.apache.kafka.common.errors.InvalidRequestException: Received request api 
> key LEADER_AND_ISR which is not enabled
> {code}
> This can be observed by running the 
> kafkatest.tests.core.zookeeper_migration_test.TestMigration.test_online_migration
>  ducktape test. Attached are logs from a local run of this test with --debug 
> set.
> It seems these errors do not affect the consistency of metadata during the 
> migration, but the excessive LISR sent out will impact performance on large 
> clusters. 



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


[jira] [Created] (KAFKA-14697) KRaft controller should not send LISR to KRaft brokers during migration

2023-02-09 Thread David Arthur (Jira)
David Arthur created KAFKA-14697:


 Summary: KRaft controller should not send LISR to KRaft brokers 
during migration
 Key: KAFKA-14697
 URL: https://issues.apache.org/jira/browse/KAFKA-14697
 Project: Kafka
  Issue Type: Bug
  Components: kraft
Affects Versions: 3.4.0
Reporter: David Arthur
 Fix For: 3.5.0, 3.4.1
 Attachments: test_online_migration.tar.gz

During a migration, after a broker is restarted in KRaft mode the KRaft 
controller is still sending it LISR as if it were a ZK broker. This results in 
errors like:

{code}
[2023-02-09 14:53:26,892] ERROR Closing socket for 
172.19.0.4:9092-172.19.0.9:32876-0 because of error (kafka.network.Processor)
org.apache.kafka.common.errors.InvalidRequestException: Received request api 
key LEADER_AND_ISR which is not enabled
{code}

This can be observed by running the 
kafkatest.tests.core.zookeeper_migration_test.TestMigration.test_online_migration
 ducktape test. Attached are logs from a local run of this test with --debug 
set.

It seems these errors do not affect the consistency of metadata during the 
migration, but the excessive LISR sent out will impact performance on large 
clusters. 



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


[jira] [Created] (KAFKA-14668) NoOpRecord causes unnecessary UMR in migration mode

2023-02-01 Thread David Arthur (Jira)
David Arthur created KAFKA-14668:


 Summary: NoOpRecord causes unnecessary UMR in migration mode
 Key: KAFKA-14668
 URL: https://issues.apache.org/jira/browse/KAFKA-14668
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur


While the KRaftMigrationDriver is in "dual write" mode, in addition to 
propagating metadata back to ZooKeeper, it is sending controller RPCs to ZK 
brokers. When a NoOpRecord is written, it will trigger the metadata 
synchronization logic and send out redundant UMR even if no metadata has 
changed. This doesn't affect correctness of the metadata in the cluster, but 
will have a significant impact on performance for larger clusters.

 

We need to avoid sending RPCs to the ZK brokers when nothing has changed.



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


[jira] [Resolved] (KAFKA-14553) RecordAccumulator hangs in infinite NOP loop

2023-01-31 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14553.
--
Resolution: Duplicate

> RecordAccumulator hangs in infinite NOP loop
> 
>
> Key: KAFKA-14553
> URL: https://issues.apache.org/jira/browse/KAFKA-14553
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 3.3.1
> Environment: - Spring Boot 3.0.1
> - Spring Cloud 2022.0.0
> Versions of dependencies are defined in boms of SB and SC:
> - micrometer-tracing-bridge-brave 1.0.0
> - zipkin-reporter-brave 2.16.3
> - zipkin-sender-kafka 2.16.3
>Reporter: Viczai Gábor
>Assignee: Luke Chen
>Priority: Minor
> Fix For: 3.4.0, 3.3.2
>
>
> *Summary:*
> There is an infinite loop in RecordAccumulator, if stickyBatchSize is 
> configured to be 0 in BuiltinPartitioner.
> (Which is the default case when using KafkaSender's default Builder.)
> *Details:*
> The infinite loop is caused by this while:
> https://github.com/apache/kafka/blob/3.3.1/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java#L293
> and this continue particularly:
> https://github.com/apache/kafka/blob/3.3.1/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java#L316
> because the partitionChanged() call in the condition always return true if 
> batchSize is 0.
> So program flow never reaches this point:
> https://github.com/apache/kafka/blob/3.3.1/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java#L318
> Thus no span data sent to Kafka ever.
> The problematic line in partitionChanged() is when it calls an update on the 
> BuiltInPartitioner:
> https://github.com/apache/kafka/blob/3.3.1/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java#L242
> which in fact always updates the partition because of this condition:
> https://github.com/apache/kafka/blob/3.3.1/clients/src/main/java/org/apache/kafka/clients/producer/internals/BuiltInPartitioner.java#L218
> therefore the next confdition in RecordAccumulator will evaluate to true also:
> https://github.com/apache/kafka/blob/3.3.1/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java#L243
> thus returning 'true' and forcing the 'continue' in the while(true) loop.
> *Suggested fix:*
> I think these conditions should be changed:
> https://github.com/apache/kafka/blob/3.3.1/clients/src/main/java/org/apache/kafka/clients/producer/internals/BuiltInPartitioner.java#L218
> The equal signs should be removed from the conditions:
> {code}if (producedBytes > stickyBatchSize && enableSwitch || producedBytes > 
> stickyBatchSize * 2) {{code}
> (Btw: line 213 also needs this modification.)
> *Note:*
> The problem arises because KafkaSender sets the batchSize to 0.
> https://github.com/openzipkin/zipkin-reporter-java/blob/2.16.3/kafka/src/main/java/zipkin2/reporter/kafka/KafkaSender.java#L88
> *Workaround:*
> Simply set the batch size greater than zero.
> {code:java}@Configuration
> public class SenderConfiguration {
>     @Bean
>     KafkaSender kafkaSender() {
>         Properties overrides = new Properties();
>         overrides.put(ProducerConfig.BATCH_SIZE_CONFIG, 1);
>         return KafkaSender.newBuilder()
>             .bootstrapServers("localhost:9092")
>             .topic("zipkin")
>             .overrides(overrides)
>             .build();
>     }
> }{code}
> *Using:*
> - Spring Boot 3.0.1
> - Spring Cloud 2022.0.0
> pom.xml (fragment):
> {code:xml}        
>             org.springframework.boot
>             spring-boot-autoconfigure
>         
>         
>             org.springframework.boot
>             spring-boot-starter-actuator
>         
>         
>             io.micrometer
>             micrometer-registry-prometheus
>         
>         
>             io.micrometer
>             micrometer-tracing-bridge-brave
>         
>         
>             io.zipkin.reporter2
>             zipkin-reporter-brave
>         
>         
>             io.zipkin.reporter2
>             zipkin-sender-kafka
>         {code}
> Everything is on default settings, except a KafkaSender is explicitely 
> created as illustrated above. (No autoconfiguration available for Kafka 
> sender.)



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


[jira] [Reopened] (KAFKA-14553) RecordAccumulator hangs in infinite NOP loop

2023-01-31 Thread David Arthur (Jira)


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

David Arthur reopened KAFKA-14553:
--

> RecordAccumulator hangs in infinite NOP loop
> 
>
> Key: KAFKA-14553
> URL: https://issues.apache.org/jira/browse/KAFKA-14553
> Project: Kafka
>  Issue Type: Bug
>  Components: clients
>Affects Versions: 3.3.1
> Environment: - Spring Boot 3.0.1
> - Spring Cloud 2022.0.0
> Versions of dependencies are defined in boms of SB and SC:
> - micrometer-tracing-bridge-brave 1.0.0
> - zipkin-reporter-brave 2.16.3
> - zipkin-sender-kafka 2.16.3
>Reporter: Viczai Gábor
>Assignee: Luke Chen
>Priority: Minor
> Fix For: 3.4.0, 3.3.2
>
>
> *Summary:*
> There is an infinite loop in RecordAccumulator, if stickyBatchSize is 
> configured to be 0 in BuiltinPartitioner.
> (Which is the default case when using KafkaSender's default Builder.)
> *Details:*
> The infinite loop is caused by this while:
> https://github.com/apache/kafka/blob/3.3.1/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java#L293
> and this continue particularly:
> https://github.com/apache/kafka/blob/3.3.1/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java#L316
> because the partitionChanged() call in the condition always return true if 
> batchSize is 0.
> So program flow never reaches this point:
> https://github.com/apache/kafka/blob/3.3.1/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java#L318
> Thus no span data sent to Kafka ever.
> The problematic line in partitionChanged() is when it calls an update on the 
> BuiltInPartitioner:
> https://github.com/apache/kafka/blob/3.3.1/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java#L242
> which in fact always updates the partition because of this condition:
> https://github.com/apache/kafka/blob/3.3.1/clients/src/main/java/org/apache/kafka/clients/producer/internals/BuiltInPartitioner.java#L218
> therefore the next confdition in RecordAccumulator will evaluate to true also:
> https://github.com/apache/kafka/blob/3.3.1/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java#L243
> thus returning 'true' and forcing the 'continue' in the while(true) loop.
> *Suggested fix:*
> I think these conditions should be changed:
> https://github.com/apache/kafka/blob/3.3.1/clients/src/main/java/org/apache/kafka/clients/producer/internals/BuiltInPartitioner.java#L218
> The equal signs should be removed from the conditions:
> {code}if (producedBytes > stickyBatchSize && enableSwitch || producedBytes > 
> stickyBatchSize * 2) {{code}
> (Btw: line 213 also needs this modification.)
> *Note:*
> The problem arises because KafkaSender sets the batchSize to 0.
> https://github.com/openzipkin/zipkin-reporter-java/blob/2.16.3/kafka/src/main/java/zipkin2/reporter/kafka/KafkaSender.java#L88
> *Workaround:*
> Simply set the batch size greater than zero.
> {code:java}@Configuration
> public class SenderConfiguration {
>     @Bean
>     KafkaSender kafkaSender() {
>         Properties overrides = new Properties();
>         overrides.put(ProducerConfig.BATCH_SIZE_CONFIG, 1);
>         return KafkaSender.newBuilder()
>             .bootstrapServers("localhost:9092")
>             .topic("zipkin")
>             .overrides(overrides)
>             .build();
>     }
> }{code}
> *Using:*
> - Spring Boot 3.0.1
> - Spring Cloud 2022.0.0
> pom.xml (fragment):
> {code:xml}        
>             org.springframework.boot
>             spring-boot-autoconfigure
>         
>         
>             org.springframework.boot
>             spring-boot-starter-actuator
>         
>         
>             io.micrometer
>             micrometer-registry-prometheus
>         
>         
>             io.micrometer
>             micrometer-tracing-bridge-brave
>         
>         
>             io.zipkin.reporter2
>             zipkin-reporter-brave
>         
>         
>             io.zipkin.reporter2
>             zipkin-sender-kafka
>         {code}
> Everything is on default settings, except a KafkaSender is explicitely 
> created as illustrated above. (No autoconfiguration available for Kafka 
> sender.)



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


[jira] [Resolved] (KAFKA-14656) Brokers rejecting LISR during ZK migration

2023-01-31 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14656.
--
Resolution: Fixed

> Brokers rejecting LISR during ZK migration
> --
>
> Key: KAFKA-14656
> URL: https://issues.apache.org/jira/browse/KAFKA-14656
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.4.0
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Blocker
> Fix For: 3.4.0
>
>
> During the ZK migration, the KRaft controller sends controller RPCs to the ZK 
> brokers (LISR, UMR, etc). Since the migration can begin immediately after a 
> ZK broker starts up with migration enabled, it is possible that this broker 
> is not seen as alive by the rest of the brokers. This is due to the KRaft 
> controller taking over before the ZK controller can send out UMR with the 
> restarted broker.
>  
> The result is that the parts of the LISR sent by KRaft immediately after the 
> metadata migration is rejected by brokers due the leader being offline. 
>  
> The fix for this is to send an UMR to all brokers after the migration with 
> the set of alive brokers.



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


[jira] [Created] (KAFKA-14656) Brokers rejecting LISR during ZK migration

2023-01-26 Thread David Arthur (Jira)
David Arthur created KAFKA-14656:


 Summary: Brokers rejecting LISR during ZK migration
 Key: KAFKA-14656
 URL: https://issues.apache.org/jira/browse/KAFKA-14656
 Project: Kafka
  Issue Type: Bug
Affects Versions: 3.4.0
Reporter: David Arthur
Assignee: David Arthur
 Fix For: 3.4.0


During the ZK migration, the KRaft controller sends controller RPCs to the ZK 
brokers (LISR, UMR, etc). Since the migration can begin immediately after a ZK 
broker starts up with migration enabled, it is possible that this broker is not 
seen as alive by the rest of the brokers. This is due to the KRaft controller 
taking over before the ZK controller can send out UMR with the restarted broker.

 

The result is that the parts of the LISR sent by KRaft immediately after the 
metadata migration is rejected by brokers due the leader being offline. 

 

The fix for this is to send an UMR to all brokers after the migration with the 
set of alive brokers.



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


[jira] [Created] (KAFKA-14621) Don't startup in migration mode if an authorizer is enabled

2023-01-13 Thread David Arthur (Jira)
David Arthur created KAFKA-14621:


 Summary: Don't startup in migration mode if an authorizer is 
enabled
 Key: KAFKA-14621
 URL: https://issues.apache.org/jira/browse/KAFKA-14621
 Project: Kafka
  Issue Type: Sub-task
Affects Versions: 3.4.0
Reporter: David Arthur
Assignee: David Arthur


In 3.4, we do not yet support migrating ACLs from ZK to KRaft. To avoid 
potential confusion and security problems, we should just disallow authorizers 
during the migration.



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


[jira] [Resolved] (KAFKA-14611) ZK broker should not send epoch during registration

2023-01-12 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14611.
--
Resolution: Fixed

> ZK broker should not send epoch during registration
> ---
>
> Key: KAFKA-14611
> URL: https://issues.apache.org/jira/browse/KAFKA-14611
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Blocker
> Fix For: 3.4.0, 3.5.0
>
>
> We need to remove the integer field from the protocol for 
> "migratingZkBrokerEpoch" and replace it with a simple boolean.



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


[jira] [Created] (KAFKA-14611) ZK broker should not send epoch during registration

2023-01-10 Thread David Arthur (Jira)
David Arthur created KAFKA-14611:


 Summary: ZK broker should not send epoch during registration
 Key: KAFKA-14611
 URL: https://issues.apache.org/jira/browse/KAFKA-14611
 Project: Kafka
  Issue Type: Sub-task
Reporter: David Arthur
Assignee: David Arthur
 Fix For: 3.4.0


We need to remove the integer field from the protocol for 
"migratingZkBrokerEpoch" and replace it with a simple boolean.



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


[jira] [Resolved] (KAFKA-14531) KRaft controller time-based snapshots are too frequent

2022-12-20 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14531.
--
Resolution: Fixed

> KRaft controller time-based snapshots are too frequent
> --
>
> Key: KAFKA-14531
> URL: https://issues.apache.org/jira/browse/KAFKA-14531
> Project: Kafka
>  Issue Type: Bug
>  Components: controller, kraft
>Affects Versions: 3.4.0
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Blocker
> Fix For: 3.4.0, 3.5.0
>
>
> The KRaft controller is generating snapshots too frequently by a factor of 
> 100. This is due to a missed conversion from milliseconds to nanoseconds.



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


[jira] [Created] (KAFKA-14531) KRaft controller time-based snapshots are too frequent

2022-12-19 Thread David Arthur (Jira)
David Arthur created KAFKA-14531:


 Summary: KRaft controller time-based snapshots are too frequent
 Key: KAFKA-14531
 URL: https://issues.apache.org/jira/browse/KAFKA-14531
 Project: Kafka
  Issue Type: Bug
  Components: controller, kraft
Affects Versions: 3.4.0
Reporter: David Arthur
Assignee: David Arthur
 Fix For: 3.4.0, 3.5.0


The KRaft controller is generating snapshots too frequently by a factor of 
100. This is due to a missed conversion from milliseconds to nanoseconds.



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


[jira] [Resolved] (KAFKA-14427) Add support for ZK migration multi-ops transaction

2022-12-15 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14427.
--
Resolution: Fixed

> Add support for ZK migration multi-ops transaction
> --
>
> Key: KAFKA-14427
> URL: https://issues.apache.org/jira/browse/KAFKA-14427
> Project: Kafka
>  Issue Type: Sub-task
>  Components: kraft
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Major
> Fix For: 3.4.0
>
>




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


[jira] [Resolved] (KAFKA-14448) ZK brokers register with KRaft during migration

2022-12-15 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14448.
--
Resolution: Fixed

> ZK brokers register with KRaft during migration
> ---
>
> Key: KAFKA-14448
> URL: https://issues.apache.org/jira/browse/KAFKA-14448
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Major
> Fix For: 3.4.0
>
>




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


[jira] [Resolved] (KAFKA-14428) Add Records, RPCs and Configs for KIP-866

2022-12-15 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14428.
--
Resolution: Fixed

> Add Records, RPCs and Configs for KIP-866
> -
>
> Key: KAFKA-14428
> URL: https://issues.apache.org/jira/browse/KAFKA-14428
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Major
> Fix For: 3.4.0
>
>




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


[jira] [Reopened] (KAFKA-14427) Add support for ZK migration multi-ops transaction

2022-12-15 Thread David Arthur (Jira)


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

David Arthur reopened KAFKA-14427:
--
  Assignee: Colin McCabe  (was: David Arthur)

> Add support for ZK migration multi-ops transaction
> --
>
> Key: KAFKA-14427
> URL: https://issues.apache.org/jira/browse/KAFKA-14427
> Project: Kafka
>  Issue Type: Sub-task
>  Components: kraft
>Reporter: David Arthur
>Assignee: Colin McCabe
>Priority: Major
> Fix For: 3.4.1
>
>




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


[jira] [Resolved] (KAFKA-14427) Add support for ZK migration multi-ops transaction

2022-12-15 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14427.
--
Resolution: Fixed

> Add support for ZK migration multi-ops transaction
> --
>
> Key: KAFKA-14427
> URL: https://issues.apache.org/jira/browse/KAFKA-14427
> Project: Kafka
>  Issue Type: Sub-task
>  Components: kraft
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Major
> Fix For: 3.4.0
>
>




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


[jira] [Created] (KAFKA-14448) ZK brokers register with KRaft during migration

2022-12-07 Thread David Arthur (Jira)
David Arthur created KAFKA-14448:


 Summary: ZK brokers register with KRaft during migration
 Key: KAFKA-14448
 URL: https://issues.apache.org/jira/browse/KAFKA-14448
 Project: Kafka
  Issue Type: Sub-task
Reporter: David Arthur
Assignee: David Arthur






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


[jira] [Created] (KAFKA-14447) Controlled shutdown for ZK brokers during migration

2022-12-07 Thread David Arthur (Jira)
David Arthur created KAFKA-14447:


 Summary: Controlled shutdown for ZK brokers during migration
 Key: KAFKA-14447
 URL: https://issues.apache.org/jira/browse/KAFKA-14447
 Project: Kafka
  Issue Type: Sub-task
Reporter: David Arthur






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


[jira] [Created] (KAFKA-14436) Initialize KRaft with arbitrary epoch

2022-12-02 Thread David Arthur (Jira)
David Arthur created KAFKA-14436:


 Summary: Initialize KRaft with arbitrary epoch
 Key: KAFKA-14436
 URL: https://issues.apache.org/jira/browse/KAFKA-14436
 Project: Kafka
  Issue Type: Sub-task
Reporter: David Arthur


For the ZK migration, we need to be able to initialize Raft with an arbitrarily 
high epoch (within the size limit). This is because during the migration, we 
want to write the Raft epoch as the controller epoch in ZK. We require that 
epochs in /controller_epoch are monotonic in order for brokers to behave 
normally. 



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


[jira] [Resolved] (KAFKA-14433) Clear all yammer metrics when test harnesses clean up

2022-12-02 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14433.
--
Resolution: Fixed

> Clear all yammer metrics when test harnesses clean up
> -
>
> Key: KAFKA-14433
> URL: https://issues.apache.org/jira/browse/KAFKA-14433
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Colin McCabe
>Assignee: David Arthur
>Priority: Major
> Attachments: image-2022-12-01-13-53-57-886.png, 
> image-2022-12-01-13-55-35-488.png
>
>
> We should clear all yammer metrics from the yammer singleton when the 
> integration test harnesses clean up. This would avoid memory leaks in tests 
> that have a lot of test cases.



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


[jira] [Created] (KAFKA-14428) Add Records, RPCs and Configs for KIP-866

2022-11-30 Thread David Arthur (Jira)
David Arthur created KAFKA-14428:


 Summary: Add Records, RPCs and Configs for KIP-866
 Key: KAFKA-14428
 URL: https://issues.apache.org/jira/browse/KAFKA-14428
 Project: Kafka
  Issue Type: Sub-task
Reporter: David Arthur
Assignee: David Arthur






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


[jira] [Created] (KAFKA-14427) Add support for ZK migration transactions

2022-11-30 Thread David Arthur (Jira)
David Arthur created KAFKA-14427:


 Summary: Add support for ZK migration transactions
 Key: KAFKA-14427
 URL: https://issues.apache.org/jira/browse/KAFKA-14427
 Project: Kafka
  Issue Type: Sub-task
  Components: kraft
Reporter: David Arthur
Assignee: David Arthur






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


[jira] [Created] (KAFKA-14305) KRaft Metadata Transactions

2022-10-14 Thread David Arthur (Jira)
David Arthur created KAFKA-14305:


 Summary: KRaft Metadata Transactions
 Key: KAFKA-14305
 URL: https://issues.apache.org/jira/browse/KAFKA-14305
 Project: Kafka
  Issue Type: New Feature
Reporter: David Arthur
 Fix For: 3.4.0


[https://cwiki.apache.org/confluence/display/KAFKA/KIP-868+Metadata+Transactions]



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


[jira] [Created] (KAFKA-14304) ZooKeeper to KRaft Migration

2022-10-14 Thread David Arthur (Jira)
David Arthur created KAFKA-14304:


 Summary: ZooKeeper to KRaft Migration
 Key: KAFKA-14304
 URL: https://issues.apache.org/jira/browse/KAFKA-14304
 Project: Kafka
  Issue Type: New Feature
Reporter: David Arthur
Assignee: David Arthur
 Fix For: 3.4.0


Top-level JIRA for 
[KIP-866|https://cwiki.apache.org/confluence/display/KAFKA/KIP-866+ZooKeeper+to+KRaft+Migration]



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


[jira] [Created] (KAFKA-14263) Investigate flaky upgrade_test.py (missing messages)

2022-09-27 Thread David Arthur (Jira)
David Arthur created KAFKA-14263:


 Summary: Investigate flaky upgrade_test.py (missing messages)
 Key: KAFKA-14263
 URL: https://issues.apache.org/jira/browse/KAFKA-14263
 Project: Kafka
  Issue Type: Test
Affects Versions: 3.3.0
Reporter: David Arthur


During system tests, we occasionally see flaky failures of upgrade_test.py. 

Here is one such recent failure on the 3.3 branch:

{code}
Module: kafkatest.tests.core.upgrade_test
Class:  TestUpgrade
Method: test_upgrade
Arguments:
{
  "compression_types": [
"none"
  ],
  "from_kafka_version": "0.9.0.1",
  "to_message_format_version": "0.9.0.1"
}
{code}

{code}
AssertionError("2 acked message did not make it to the Consumer. They are: 
[21017, 21014]. We validated that the first 2 of these missing messages 
correctly made it into Kafka's data files. This suggests they were lost on 
their way to the consumer.")
{code}

It would be good to try and reproduce these types of failures and see how we 
can make the test more robust.



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


[jira] [Resolved] (KAFKA-14214) StandardAuthorizer may transiently process ACLs out of write order

2022-09-20 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14214.
--
Resolution: Fixed

> StandardAuthorizer may transiently process ACLs out of write order
> --
>
> Key: KAFKA-14214
> URL: https://issues.apache.org/jira/browse/KAFKA-14214
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.3
>Reporter: Akhilesh Chaganti
>Assignee: Akhilesh Chaganti
>Priority: Blocker
> Fix For: 3.3.0
>
>
> The issue with StandardAuthorizer#authorize is, that it looks up 
> aclsByResources (which is of type ConcurrentSkipListMap)twice for every 
> authorize call and uses Iterator with weak consistency guarantees on top of 
> aclsByResources. This can cause the authorize function call to process the 
> concurrent writes out of order.
> *Issue 1:*
> When StandardAuthorizer calls into a simple authorize function, we check the 
> ACLs for literal/prefix matches for the resource and then make one more call 
> to check the ACLs for matching wildcard entries. Between the two 
> (checkSection) calls, let’s assume we add a DENY for resource literal and add 
> an ALLOW ALL wildcard. The first call to check literal/prefix rules will SKIP 
> DENY ACL since the writes are not yet processed and the second call would 
> find ALLOW wildcard entry which results in ALLOW authorization for the 
> resource when it is actually DENY.
> *Issue: 2*
> For authorization, StandardAuthorizer depends on an iterator that iterates 
> through the ordered set of ACLs. The iterator has weak consistency 
> guarantees. So when writes for two ACLs occur, one of the ACLs might be still 
> visible to the iterator while the other is not. 
> Let’s say below two ACLS are getting added in the following order to the set.
> Acl1 = StandardAcl(TOPIC, foobar, LITERAL, DENY, READ, user1)
> Acl2 = StandardAcl(TOPIC, foo, PREFIX, ALLOW, READ, user1)
> Depending on the position of the iterator on the ordered set during the write 
> call, the iterator might just see Acl2 which prompts it to ALLOW the topic to 
> be READ even though the DENY rule was written before.



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


[jira] [Resolved] (KAFKA-14243) Temporarily disable unsafe downgrade

2022-09-20 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14243.
--
Resolution: Fixed

> Temporarily disable unsafe downgrade
> 
>
> Key: KAFKA-14243
> URL: https://issues.apache.org/jira/browse/KAFKA-14243
> Project: Kafka
>  Issue Type: Bug
>Reporter: Colin McCabe
>Assignee: Colin McCabe
>Priority: Blocker
> Fix For: 3.3.0
>
>
> Temporarily disable unsafe downgrade since we haven't implemented reloading 
> snapshots on unsafe downgrade



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


[jira] [Created] (KAFKA-14186) Add unit tests for BatchFileWriter

2022-08-27 Thread David Arthur (Jira)
David Arthur created KAFKA-14186:


 Summary: Add unit tests for BatchFileWriter
 Key: KAFKA-14186
 URL: https://issues.apache.org/jira/browse/KAFKA-14186
 Project: Kafka
  Issue Type: Test
Reporter: David Arthur
 Fix For: 3.4.0


We have integration tests that cover this class, but no direct unit tests



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


[jira] [Created] (KAFKA-14178) NoOpRecord incorrectly causes high controller queue time metric

2022-08-23 Thread David Arthur (Jira)
David Arthur created KAFKA-14178:


 Summary: NoOpRecord incorrectly causes high controller queue time 
metric
 Key: KAFKA-14178
 URL: https://issues.apache.org/jira/browse/KAFKA-14178
 Project: Kafka
  Issue Type: Bug
  Components: controller, kraft, metrics
Reporter: David Arthur
 Fix For: 3.3.0


When a deferred event is added to the queue in ControllerQuorum, we include the 
total time it sat in the queue as part of the "EventQueueTimeMs" metric in 
QuorumControllerMetrics.

With the introduction of NoOpRecords, the p99 value for this metric is equal to 
the frequency that we schedule the no-op records. E.g., if no-op records are 
scheduled every 5 seconds, we will see p99 EventQueueTimeMs of 5 seconds.

This makes it difficult (impossible) to see if there is some delay in the event 
processing on the controller.




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


[jira] [Resolved] (KAFKA-13410) KRaft Upgrades

2022-08-22 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-13410.
--
Resolution: Fixed

The unfinished tasks from this issue were moved to KAFKA-14175

> KRaft Upgrades
> --
>
> Key: KAFKA-13410
> URL: https://issues.apache.org/jira/browse/KAFKA-13410
> Project: Kafka
>  Issue Type: New Feature
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Major
> Fix For: 3.3.0
>
>
> This is the placeholder JIRA for KIP-778



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


[jira] [Resolved] (KAFKA-13935) Factor out static IBP usages from broker

2022-08-22 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-13935.
--
  Assignee: David Arthur
Resolution: Fixed

> Factor out static IBP usages from broker
> 
>
> Key: KAFKA-13935
> URL: https://issues.apache.org/jira/browse/KAFKA-13935
> Project: Kafka
>  Issue Type: Sub-task
>Affects Versions: 3.3.0
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Major
>
> We pass the IBP down to the log layer for checking things like compression 
> support. Currently, we are still reading this from KafkaConfig. In ZK mode 
> this is fine, but in KRaft mode, reading the IBP from the config is not 
> supported.
> Since KRaft only supports IBP/MetadataVersion greater than 3.0 (which 
> supports the compression mode we check for), we may be able to avoid using a 
> dynamic call and/or volatile to get the current version. 



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


[jira] [Created] (KAFKA-14175) KRaft Upgrades Part 2

2022-08-22 Thread David Arthur (Jira)
David Arthur created KAFKA-14175:


 Summary: KRaft Upgrades Part 2
 Key: KAFKA-14175
 URL: https://issues.apache.org/jira/browse/KAFKA-14175
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur
Assignee: David Arthur
 Fix For: 3.4.0


This is the parent issue for KIP-778 tasks which were not completed for the 3.3 
release.



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


[jira] [Resolved] (KAFKA-14039) Fix KRaft AlterConfigPolicy usage

2022-08-15 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14039.
--
Resolution: Fixed

> Fix KRaft AlterConfigPolicy usage
> -
>
> Key: KAFKA-14039
> URL: https://issues.apache.org/jira/browse/KAFKA-14039
> Project: Kafka
>  Issue Type: Bug
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Major
> Fix For: 3.3.0, 3.4.0
>
>
> In ConfigurationControlManager, we are currently passing all the 
> configuration values known to the controller down into the AlterConfigPolicy. 
> This does not match the behavior in ZK mode where we only pass configs which 
> were included in the alter configs request.
> This can lead to different unexpected behavior in custom AlterConfigPolicy 
> implementations



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


[jira] [Resolved] (KAFKA-14115) Password configs are logged in plaintext in KRaft

2022-08-04 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14115.
--
  Assignee: David Arthur  (was: Prem Kamal)
Resolution: Fixed

> Password configs are logged in plaintext in KRaft
> -
>
> Key: KAFKA-14115
> URL: https://issues.apache.org/jira/browse/KAFKA-14115
> Project: Kafka
>  Issue Type: Bug
>  Components: kraft
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Critical
> Fix For: 3.3.0, 3.4.0, 3.2.2
>
>
> While investigating KAFKA-14111, I also noticed that 
> ConfigurationControlManager is logging sensitive configs in plaintext at INFO 
> level.
> {code}
> [2022-07-27 12:14:09,927] INFO [Controller 1] ConfigResource(type=BROKER, 
> name='1'): set configuration listener.name.external.ssl.key.password to bar 
> (org.apache.kafka.controller.ConfigurationControlManager)
> {code}
> Once this new config reaches the broker, it is logged again, but this time it 
> is redacted
> {code}
> [2022-07-27 12:14:09,957] INFO [BrokerMetadataPublisher id=1] Updating broker 
> 1 with new configuration : listener.name.external.ssl.key.password -> 
> [hidden] (kafka.server.metadata.BrokerMetadataPublisher)
> {code}



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


[jira] [Resolved] (KAFKA-14136) AlterConfigs in KRaft does not generate records for unchanged values

2022-08-04 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14136.
--
Resolution: Fixed

> AlterConfigs in KRaft does not generate records for unchanged values
> 
>
> Key: KAFKA-14136
> URL: https://issues.apache.org/jira/browse/KAFKA-14136
> Project: Kafka
>  Issue Type: Bug
>  Components: kraft
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Major
> Fix For: 3.3.0, 3.4.0, 3.2.2
>
>
> In ZK, when handling LegacyAlterConfigs or IncrementalAlterConfigs, we call 
> certain code paths regardless of what values are included in the request. We 
> utilize this behavior to force a broker to reload a keystore or truststore 
> from disk (we sent an AlterConfig with the keystore path unchanged).
> In KRaft, however, we have an optimization to only generate ConfigRecords if 
> the incoming AtlerConfig request will result in actual config changes. This 
> means the broker never receives any records for "no-op" config changes and we 
> cannot trigger certain code paths. 



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


[jira] [Created] (KAFKA-14136) AlterConfigs in KRaft does not generate records for unchanged values

2022-08-03 Thread David Arthur (Jira)
David Arthur created KAFKA-14136:


 Summary: AlterConfigs in KRaft does not generate records for 
unchanged values
 Key: KAFKA-14136
 URL: https://issues.apache.org/jira/browse/KAFKA-14136
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur
Assignee: David Arthur
 Fix For: 3.3.0, 3.4.0


In ZK, when handling LegacyAlterConfigs or IncrementalAlterConfigs, we call 
certain code paths regardless of what values are included in the request. We 
utilize this behavior to force a broker to reload a keystore or truststore from 
disk (we sent an AlterConfig with the keystore path unchanged).

In KRaft, however, we have an optimization to only generate ConfigRecords if 
the incoming AtlerConfig request will result in actual config changes. This 
means the broker never receives any records for "no-op" config changes and we 
cannot trigger certain code paths. 




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


[jira] [Resolved] (KAFKA-14111) Dynamic config update fails for "password" configs in KRaft

2022-08-03 Thread David Arthur (Jira)


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

David Arthur resolved KAFKA-14111.
--
Resolution: Fixed

> Dynamic config update fails for "password" configs in KRaft
> ---
>
> Key: KAFKA-14111
> URL: https://issues.apache.org/jira/browse/KAFKA-14111
> Project: Kafka
>  Issue Type: Bug
>  Components: kraft
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Major
> Fix For: 3.3.0, 3.4.0
>
>
> Two related bugs found when working on updating 
> DynamicBrokerReconfigurationTest for KRaft. 
> Firstly, if we issue an AlterConfig (or IncrementalAlterConfig) for a broker 
> config that is defined as a "password", it will succeed on the controller, 
> but throw an error when the broker handles it. 
> For example, on a vanilla cluster running "config/kraft/server.properties"
> {code}
> /bin/kafka-configs.sh --bootstrap-server localhost:9092  --alter --broker 1 
> --add-config listener.name.external.ssl.key.password=foo 
> {code}
> results in
> {code}
> [2022-07-26 16:24:05,049] ERROR Dynamic password config 
> listener.name.external.ssl.key.password could not be decoded, ignoring. 
> (kafka.server.DynamicBrokerConfig)
> org.apache.kafka.common.config.ConfigException: Password encoder secret not 
> configured
>   at 
> kafka.server.DynamicBrokerConfig.$anonfun$passwordEncoder$1(DynamicBrokerConfig.scala:352)
>   at scala.Option.getOrElse(Option.scala:201)
>   at 
> kafka.server.DynamicBrokerConfig.passwordEncoder(DynamicBrokerConfig.scala:352)
>   at 
> kafka.server.DynamicBrokerConfig.decodePassword$1(DynamicBrokerConfig.scala:393)
>   at 
> kafka.server.DynamicBrokerConfig.$anonfun$fromPersistentProps$5(DynamicBrokerConfig.scala:404)
>   at 
> kafka.server.DynamicBrokerConfig.$anonfun$fromPersistentProps$5$adapted(DynamicBrokerConfig.scala:402)
>   at 
> kafka.utils.Implicits$MapExtensionMethods$.$anonfun$forKeyValue$1(Implicits.scala:62)
>   at scala.collection.MapOps.foreachEntry(Map.scala:244)
>   at scala.collection.MapOps.foreachEntry$(Map.scala:240)
>   at scala.collection.AbstractMap.foreachEntry(Map.scala:405)
>   at 
> kafka.server.DynamicBrokerConfig.fromPersistentProps(DynamicBrokerConfig.scala:402)
>   at 
> kafka.server.DynamicBrokerConfig.$anonfun$updateBrokerConfig$1(DynamicBrokerConfig.scala:300)
>   at 
> kafka.server.DynamicBrokerConfig.updateBrokerConfig(DynamicBrokerConfig.scala:299)
>   at 
> kafka.server.BrokerConfigHandler.processConfigChanges(ConfigHandler.scala:221)
>   at 
> kafka.server.metadata.BrokerMetadataPublisher.$anonfun$publish$15(BrokerMetadataPublisher.scala:212)
>   at java.base/java.util.HashMap$KeySet.forEach(HashMap.java:1008)
>   at 
> kafka.server.metadata.BrokerMetadataPublisher.$anonfun$publish$14(BrokerMetadataPublisher.scala:190)
>   at 
> kafka.server.metadata.BrokerMetadataPublisher.$anonfun$publish$14$adapted(BrokerMetadataPublisher.scala:189)
>   at scala.Option.foreach(Option.scala:437)
>   at 
> kafka.server.metadata.BrokerMetadataPublisher.publish(BrokerMetadataPublisher.scala:189)
>   at 
> kafka.server.metadata.BrokerMetadataListener.kafka$server$metadata$BrokerMetadataListener$$publish(BrokerMetadataListener.scala:293)
>   at 
> kafka.server.metadata.BrokerMetadataListener$HandleCommitsEvent.$anonfun$run$2(BrokerMetadataListener.scala:126)
>   at 
> kafka.server.metadata.BrokerMetadataListener$HandleCommitsEvent.$anonfun$run$2$adapted(BrokerMetadataListener.scala:126)
>   at scala.Option.foreach(Option.scala:437)
>   at 
> kafka.server.metadata.BrokerMetadataListener$HandleCommitsEvent.run(BrokerMetadataListener.scala:126)
>   at 
> org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:121)
>   at 
> org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:200)
>   at 
> org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:173)
>   at java.base/java.lang.Thread.run(Thread.java:833)
> {code}.
> If a {{password.encoder.secret}} is supplied, this still fails but with:
> {code}
> [2022-07-26 16:27:23,247] ERROR Dynamic password config 
> listener.name.external.ssl.key.password could not be decoded, ignoring. 
> (kafka.server.DynamicBrokerConfig)
> java.lang.StringIndexOutOfBoundsException: begin 0, end -1, length 3
>   at java.base/java.lang.String.checkBoundsBeginEnd(String.java:4604)
>   at java.base/java.lang.String.substring(String.java:2707)
>   at kafka.utils.CoreUtils$.$anonfun$parseCsvMap$1(CoreUtils.scala:173)
>   at scala.collection.ArrayOps$.map$extension(ArrayOps.scala:929)
>   at kafka.utils.CoreUtils$.parseCsvMap(CoreUtils.scala:171)
>   at 

[jira] [Created] (KAFKA-14126) Convert remaining DynamicBrokerReconfigurationTest tests to KRaft

2022-07-29 Thread David Arthur (Jira)
David Arthur created KAFKA-14126:


 Summary: Convert remaining DynamicBrokerReconfigurationTest tests 
to KRaft
 Key: KAFKA-14126
 URL: https://issues.apache.org/jira/browse/KAFKA-14126
 Project: Kafka
  Issue Type: Test
Reporter: David Arthur


After the initial conversion in https://github.com/apache/kafka/pull/12455, 
three tests still need to be converted. 

* testKeyStoreAlter
* testTrustStoreAlter
* testThreadPoolResize





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


[jira] [Created] (KAFKA-14115) Password configs are logged in plaintext in KRaft

2022-07-27 Thread David Arthur (Jira)
David Arthur created KAFKA-14115:


 Summary: Password configs are logged in plaintext in KRaft
 Key: KAFKA-14115
 URL: https://issues.apache.org/jira/browse/KAFKA-14115
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur
 Fix For: 3.3.0, 3.4.0


While investigating KAFKA-14111, I also noticed that 
ConfigurationControlManager is logging sensitive configs in plaintext at INFO 
level.


{code}
[2022-07-27 12:14:09,927] INFO [Controller 1] ConfigResource(type=BROKER, 
name='1'): set configuration listener.name.external.ssl.key.password to bar 
(org.apache.kafka.controller.ConfigurationControlManager)
{code}

Once this new config reaches the broker, it is logged again, but this time it 
is redacted

{code}
[2022-07-27 12:14:09,957] INFO [BrokerMetadataPublisher id=1] Updating broker 1 
with new configuration : listener.name.external.ssl.key.password -> [hidden] 
(kafka.server.metadata.BrokerMetadataPublisher)
{code}



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


[jira] [Created] (KAFKA-14111) Dynamic config update fails for "password" configs in KRaft

2022-07-26 Thread David Arthur (Jira)
David Arthur created KAFKA-14111:


 Summary: Dynamic config update fails for "password" configs in 
KRaft
 Key: KAFKA-14111
 URL: https://issues.apache.org/jira/browse/KAFKA-14111
 Project: Kafka
  Issue Type: Bug
Reporter: David Arthur


Two related bugs found when working on updating 
DynamicBrokerReconfigurationTest for KRaft. 

Firstly, if we issue an AlterConfig (or IncrementalAlterConfig) for a broker 
config that is defined as a "password", it will succeed on the controller, but 
throw an error when the broker handles it. 

For example, on a vanilla cluster running "config/kraft/server.properties"

{code}
/bin/kafka-configs.sh --bootstrap-server localhost:9092  --alter --broker 1 
--add-config listener.name.external.ssl.key.password=foo 
{code}

results in

{code}
[2022-07-26 16:24:05,049] ERROR Dynamic password config 
listener.name.external.ssl.key.password could not be decoded, ignoring. 
(kafka.server.DynamicBrokerConfig)
org.apache.kafka.common.config.ConfigException: Password encoder secret not 
configured
at 
kafka.server.DynamicBrokerConfig.$anonfun$passwordEncoder$1(DynamicBrokerConfig.scala:352)
at scala.Option.getOrElse(Option.scala:201)
at 
kafka.server.DynamicBrokerConfig.passwordEncoder(DynamicBrokerConfig.scala:352)
at 
kafka.server.DynamicBrokerConfig.decodePassword$1(DynamicBrokerConfig.scala:393)
at 
kafka.server.DynamicBrokerConfig.$anonfun$fromPersistentProps$5(DynamicBrokerConfig.scala:404)
at 
kafka.server.DynamicBrokerConfig.$anonfun$fromPersistentProps$5$adapted(DynamicBrokerConfig.scala:402)
at 
kafka.utils.Implicits$MapExtensionMethods$.$anonfun$forKeyValue$1(Implicits.scala:62)
at scala.collection.MapOps.foreachEntry(Map.scala:244)
at scala.collection.MapOps.foreachEntry$(Map.scala:240)
at scala.collection.AbstractMap.foreachEntry(Map.scala:405)
at 
kafka.server.DynamicBrokerConfig.fromPersistentProps(DynamicBrokerConfig.scala:402)
at 
kafka.server.DynamicBrokerConfig.$anonfun$updateBrokerConfig$1(DynamicBrokerConfig.scala:300)
at 
kafka.server.DynamicBrokerConfig.updateBrokerConfig(DynamicBrokerConfig.scala:299)
at 
kafka.server.BrokerConfigHandler.processConfigChanges(ConfigHandler.scala:221)
at 
kafka.server.metadata.BrokerMetadataPublisher.$anonfun$publish$15(BrokerMetadataPublisher.scala:212)
at java.base/java.util.HashMap$KeySet.forEach(HashMap.java:1008)
at 
kafka.server.metadata.BrokerMetadataPublisher.$anonfun$publish$14(BrokerMetadataPublisher.scala:190)
at 
kafka.server.metadata.BrokerMetadataPublisher.$anonfun$publish$14$adapted(BrokerMetadataPublisher.scala:189)
at scala.Option.foreach(Option.scala:437)
at 
kafka.server.metadata.BrokerMetadataPublisher.publish(BrokerMetadataPublisher.scala:189)
at 
kafka.server.metadata.BrokerMetadataListener.kafka$server$metadata$BrokerMetadataListener$$publish(BrokerMetadataListener.scala:293)
at 
kafka.server.metadata.BrokerMetadataListener$HandleCommitsEvent.$anonfun$run$2(BrokerMetadataListener.scala:126)
at 
kafka.server.metadata.BrokerMetadataListener$HandleCommitsEvent.$anonfun$run$2$adapted(BrokerMetadataListener.scala:126)
at scala.Option.foreach(Option.scala:437)
at 
kafka.server.metadata.BrokerMetadataListener$HandleCommitsEvent.run(BrokerMetadataListener.scala:126)
at 
org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:121)
at 
org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:200)
at 
org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:173)
at java.base/java.lang.Thread.run(Thread.java:833)
{code}.


If a {{password.encoder.secret}} is supplied, this still fails but with:


{code}
[2022-07-26 16:27:23,247] ERROR Dynamic password config 
listener.name.external.ssl.key.password could not be decoded, ignoring. 
(kafka.server.DynamicBrokerConfig)
java.lang.StringIndexOutOfBoundsException: begin 0, end -1, length 3
at java.base/java.lang.String.checkBoundsBeginEnd(String.java:4604)
at java.base/java.lang.String.substring(String.java:2707)
at kafka.utils.CoreUtils$.$anonfun$parseCsvMap$1(CoreUtils.scala:173)
at scala.collection.ArrayOps$.map$extension(ArrayOps.scala:929)
at kafka.utils.CoreUtils$.parseCsvMap(CoreUtils.scala:171)
at kafka.utils.PasswordEncoder.decode(PasswordEncoder.scala:88)
at 
kafka.server.DynamicBrokerConfig.decodePassword$1(DynamicBrokerConfig.scala:393)
at 
kafka.server.DynamicBrokerConfig.$anonfun$fromPersistentProps$5(DynamicBrokerConfig.scala:404)
at 
kafka.server.DynamicBrokerConfig.$anonfun$fromPersistentProps$5$adapted(DynamicBrokerConfig.scala:402)
at 

  1   2   3   >