[
https://issues.apache.org/jira/browse/KAFKA-16320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17842867#comment-17842867
]
Emanuele Sabellico commented on KAFKA-16320:
--------------------------------------------
Corresponding code in AK when using ZK
https://github.com/apache/kafka/blob/8d914b543d5d23031fe178d424f45789eaa8d1fc/core/src/main/scala/kafka/server/ZkAdminManager.scala#L238
You see that in that case the CreateTopics operation is executed and the
response is immediately returned without awaiting it if timeout <= 0. Don't
know if it was for a bug or it was the expected behaviour.
> CreateTopics, DeleteTopics and CreatePartitions differences between Zookeeper
> and KRaft
> ---------------------------------------------------------------------------------------
>
> Key: KAFKA-16320
> URL: https://issues.apache.org/jira/browse/KAFKA-16320
> Project: Kafka
> Issue Type: Bug
> Affects Versions: 3.7.0
> Reporter: Emanuele Sabellico
> Assignee: Chia-Ping Tsai
> Priority: Minor
>
> Test number 0081 with these operations is failing in librdkafka when using
> KRaft but not when using Zookeeper. The test sets the operation timeout to 0
> and expects that those operations are executed asynchronously. The returned
> err was REQUEST_TIMED_OUT and it was converted to NO_ERROR if operation
> timeout is <= 0.
> With KRaft instead NO_ERROR is returned, but the topics aren't created or
> deleted.
> Also passing an invalid configuration option it's returning NO_ERROR instead
> of INVALID_CONFIG, that is what happens in Zookeeper or with KRaft if
> operation timeout is > 0.
> https://github.com/confluentinc/librdkafka/blob/a6d85bdbc1023b1a5477b8befe516242c3e182f6/tests/0081-admin.c#L5174C9-L5174C29
> {code:java}
> /* For non-blocking CreateTopicsRequests the broker
> * will returned REQUEST_TIMED_OUT for topics
> * that were triggered for creation -
> * we hide this error code from the application
> * since the topic creation is in fact in progress. */
> if (error_code == RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT &&
> rd_kafka_confval_get_int(&rko_req->rko_u.admin_request
> .options.operation_timeout) <=
> 0) {
> error_code = RD_KAFKA_RESP_ERR_NO_ERROR;
> this_errstr = NULL;
> }
> {code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)