[ 
https://issues.apache.org/jira/browse/KAFKA-17993?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17897624#comment-17897624
 ] 

Edoardo Comar edited comment on KAFKA-17993 at 11/12/24 3:59 PM:
-----------------------------------------------------------------

one topic had 5000 partitions and the IncrementalAlterConfigRequest for 
{color:#00627a}modifyTopicThrottles {color}looks like :

 

{{{color:#000000}AlterConfigOp{opType=SET, 
configEntry=ConfigEntry(name=leader.replication.throttled.replicas, {color}}}
{{{color:#000000}value=0:0,0:1,0:2,1000:0,1000:1,1000:2,1001:0,1001:1,1001:2,1002:0,1002:1,1002:2,1003:0,1003:1,1003:2,1004:0,1004:1,1004:2,....{color}
 }}

 

and the value was over 32k so could not be serialized

 


was (Author: ecomar):
one topic had 5000 partitions and the IncrementalAlterConfigRequest looks like :

 

{{{color:#000000}AlterConfigOp{opType=SET, 
configEntry=ConfigEntry(name=leader.replication.throttled.replicas, {color}}}
{{{color:#000000}value=0:0,0:1,0:2,1000:0,1000:1,1000:2,1001:0,1001:1,1001:2,1002:0,1002:1,1002:2,1003:0,1003:1,1003:2,1004:0,1004:1,1004:2,....{color}
 }}

 

and the value was over 32k so could not be serialized

 

> reassign partition tool stuck with uncaught exception: 'value' field is too 
> long to be serialized
> -------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-17993
>                 URL: https://issues.apache.org/jira/browse/KAFKA-17993
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients
>            Reporter: Edoardo Comar
>            Priority: Major
>
> Running the reassignment script for about 5800 partitions, with both throttle 
> options being set, the tool remained stuck with this exception
> {{ERROR Uncaught exception in thread 'kafka-admin-client-thread | 
> reassign-partitions-tool': (org.apache.kafka.common.utils.KafkaThread)}}
> {{java.lang.RuntimeException: 'value' field is too long to be serialized}}
> {{    at 
> org.apache.kafka.common.message.IncrementalAlterConfigsRequestData$AlterableConfig.addSize(IncrementalAlterConfigsRequestData.java:776)}}
> {{    at 
> org.apache.kafka.common.message.IncrementalAlterConfigsRequestData$AlterConfigsResource.addSize(IncrementalAlterConfigsRequestData.java:463)}}
> {{    at 
> org.apache.kafka.common.message.IncrementalAlterConfigsRequestData.addSize(IncrementalAlterConfigsRequestData.java:187)}}
> {{    at 
> org.apache.kafka.common.protocol.SendBuilder.buildSend(SendBuilder.java:218)}}
> {{    at 
> org.apache.kafka.common.protocol.SendBuilder.buildRequestSend(SendBuilder.java:187)}}
> {{    at 
> org.apache.kafka.common.requests.AbstractRequest.toSend(AbstractRequest.java:108)}}
> {{    at 
> org.apache.kafka.clients.NetworkClient.doSend(NetworkClient.java:535)}}
> {{    at 
> org.apache.kafka.clients.NetworkClient.doSend(NetworkClient.java:511)}}
> {{    at org.apache.kafka.clients.NetworkClient.send(NetworkClient.java:471)}}
> {{    at 
> org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.sendEligibleCalls(KafkaAdminClient.java:1156)}}
> {{    at 
> org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.processRequests(KafkaAdminClient.java:1369)}}
> {{    at 
> org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.run(KafkaAdminClient.java:1312)}}
> {{    at java.base/java.lang.Thread.run(Unknown Source)}}
>  
> The same json file previously passed the --verify step



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

Reply via email to