lucasbru commented on code in PR #19544:
URL: https://github.com/apache/kafka/pull/19544#discussion_r2057854972


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java:
##########
@@ -521,11 +521,14 @@ private void onSuccessResponse(final 
StreamsGroupHeartbeatResponse response, fin
         }
 
         List<StreamsGroupHeartbeatResponseData.Status> statuses = 
data.status();
-        if (statuses != null && !statuses.isEmpty()) {
-            String statusDetails = statuses.stream()
-                .map(status -> "(" + status.statusCode() + ") " + 
status.statusDetail())
-                .collect(Collectors.joining(", "));
-            logger.warn("Membership is in the following statuses: {}", 
statusDetails);
+        if (statuses != null) {
+            streamsRebalanceData.setStatuses(statuses);
+            if (!statuses.isEmpty()) {
+                String statusDetails = statuses.stream()
+                    .map(status -> "(" + status.statusCode() + ") " + 
status.statusDetail())
+                    .collect(Collectors.joining(", "));
+                logger.warn("Membership is in the following statuses: {}", 
statusDetails);

Review Comment:
   It's a good suggestion, that I was thinking about as well. The thing is, 
right now the streams application keeps running when a source topic is missing, 
and this membership log is the only client-side thing that is pointing out why 
no processing is happening. So I would like to keep it at `WARN` for now. It 
was intended in the KIP to be like this, but I discussed with Matthias as well, 
that we want to fail in these cases instead - so if a source topic doesn't 
exist, internal topics aren't created within a certain time, or topics are 
malpartitioned, we actually want to throw an exception (inside StreamThread). 
This PR is preparing the change by propagating the status to the StreamThread.
   
   Lets make this an `INFO` log, once we treat the statuses more explicitly 
inside `StreamThread`.



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