apoorvmittal10 commented on code in PR #19998:
URL: https://github.com/apache/kafka/pull/19998#discussion_r2157024298


##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -501,8 +501,12 @@ public CompletableFuture<Void> maybeInitialize() {
 
                 if (partitionData.errorCode() != Errors.NONE.code()) {
                     KafkaException ex = 
fetchPersisterError(partitionData.errorCode(), partitionData.errorMessage());
-                    log.error("Failed to initialize the share partition: 
{}-{}. Exception occurred: {}.",
+                    log.debug("Failed to initialize the share partition: 
{}-{}. Exception occurred: {}.",
                         groupId, topicIdPartition, partitionData);
+                    if (!(ex instanceof UnknownServerException)) {

Review Comment:
   This doesn't seems to be correct, so if the error code is 13 then only we 
should not log the eror message. Example log from persister.
   
   ```
   Failed to initialize the share partition: <.......>. Exception occurred: 
PartitionData(partition=144,stateEpoch=0,startOffset=0,errorCode=13,errorMessage=Error
 in read state RPC. The server disconnected before a response was 
received.,leaderEpoch=0,stateBatches=[]).



##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -2324,8 +2328,12 @@ CompletableFuture<Void> 
writeShareGroupState(List<PersisterStateBatch> stateBatc
                 PartitionErrorData partitionData = state.partitions().get(0);
                 if (partitionData.errorCode() != Errors.NONE.code()) {
                     KafkaException ex = 
fetchPersisterError(partitionData.errorCode(), partitionData.errorMessage());
-                    log.error("Failed to write the share group state for share 
partition: {}-{} due to exception",
+                    log.debug("Failed to write the share group state for share 
partition: {}-{} due to exception",
                         groupId, topicIdPartition, ex);
+                    if (!(ex instanceof UnknownServerException)) {

Review Comment:
   Something similar to above but here the log line in SharePartition shows: 
`org.apache.kafka.common.errors.UnknownServerException` rather should Persister 
send NetworkException as like in ReadAPI...?



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

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

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

Reply via email to