jolshan commented on code in PR #16443:
URL: https://github.com/apache/kafka/pull/16443#discussion_r1761278964


##########
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java:
##########
@@ -2305,11 +2306,23 @@ public CompletableFuture<UpdateFeaturesResponseData> 
updateFeatures(
         }).thenApply(result -> {
             UpdateFeaturesResponseData responseData = new 
UpdateFeaturesResponseData();
             responseData.setResults(new 
UpdateFeaturesResponseData.UpdatableFeatureResultCollection(result.size()));
-            result.forEach((featureName, error) -> responseData.results().add(
-                new UpdateFeaturesResponseData.UpdatableFeatureResult()
-                    .setFeature(featureName)
-                    .setErrorCode(error.error().code())
-                    .setErrorMessage(error.message())));
+            List<String> featuresWithErrors = new ArrayList<>();
+            result.forEach((featureName, error) -> {
+                if (!error.error().equals(Errors.NONE))  {
+                    featuresWithErrors.add(featureName + ":" + 
error.error().exceptionName() + " (" + error.message() + ")");
+                }
+                responseData.results().add(
+                    new UpdateFeaturesResponseData.UpdatableFeatureResult()
+                        .setFeature(featureName)
+                        .setErrorCode(error.error().code())
+                        .setErrorMessage(error.message()));
+            });
+            // If the request is a newer version, indicate the update failed 
with a top level error if any update failed.
+            if (context.requestHeader().requestApiVersion() > 1 && 
featuresWithErrors.size() > 0) {

Review Comment:
   I guess the other question I have is whether it is better to fail fast or 
potentially show what other errors could be. 
   For example, if one feature is invalid because it not a real feature and one 
is invalid because of a dependency, should we fail with just the not real 
feature error? Or should we include both in the error message? 
   
   It is a little annoying to have to rerun the command for each error, but it 
could also be confusing if the error code is not the same for the different 
errors and we set them all to be the same.
   
   Perhaps failing fast with one error at at time is the best option at the 
expense of the user having to rerun the command for each error. In that case, 
we can also change the error message logic to just share the error of the one 
error we found.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to