jeffkbkim commented on code in PR #13870:
URL: https://github.com/apache/kafka/pull/13870#discussion_r1258924230


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +282,32 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return 
FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new 
CompletableFuture<>();
+
+        if (!isGroupIdNotEmpty(request.groupId())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleWriteOperation("generic-group-join",
+            topicPartitionFor(request.groupId()),
+            coordinator -> coordinator.genericGroupJoin(context, request, 
responseFuture)
+        ).exceptionally(exception -> {
+            log.error("Request {} hit an unexpected exception: {}",
+                request, exception.getMessage());

Review Comment:
   this would log all errors while appending/committing and if 
`generateRecordsAndResponse` throws an unexpected exception. shouldn't we log 
them? 
   
   it doesn't seem like we do for `consumerGroupHeartbeat()` -- maybe just 
filter out the coordinator not available / not coordinator error codes?



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