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


##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -2575,6 +2578,13 @@ private List<AcquiredRecords> 
filterAbortedTransactionalAcquiredRecords(
         return filterRecordBatchesFromAcquiredRecords(acquiredRecords, 
recordsToArchive);
     }
 
+    private void logError(String message, Throwable e) {
+        log.debug(message, e);
+        if (!(e instanceof NetworkException)) {
+            log.error(message, e);
+        }

Review Comment:
   If not NetworkException then same error message will be twice in debug mode:
   ```suggestion
           if (e instanceof NetworkException) {
               log.debug(message, e);
           } else {
                log.error(message, e);
           }
   ```



##########
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:
   Make sense, just 2 minor change needed as well, move below line to debug or 
trace as they call `writeShareGroupState` which logs error message 
appropiratley hence these exception messages should go in debug or trace:
   
   
https://github.com/apache/kafka/blob/659ace836caa3abc59f73ef2c24d45dfc6fa30ff/core/src/main/java/kafka/server/share/SharePartition.java#L2079
   
   
https://github.com/apache/kafka/blob/659ace836caa3abc59f73ef2c24d45dfc6fa30ff/core/src/main/java/kafka/server/share/SharePartition.java#L2426



##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -2324,8 +2325,8 @@ 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",
-                        groupId, topicIdPartition, ex);
+                    logError(String.format("Failed to write the share group 
state for share partition: %s-%s due to exception",
+                        groupId, topicIdPartition), ex);

Review Comment:
   nit: for my understanding. What difference in log do we get by String.format 
now?



##########
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:
   Make sense, thanks.



-- 
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