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


##########
clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java:
##########
@@ -82,17 +84,31 @@ public static UpdateFeaturesResponse parse(ByteBuffer 
buffer, short version) {
         return new UpdateFeaturesResponse(new UpdateFeaturesResponseData(new 
ByteBufferAccessor(buffer), version));
     }
 
-    public static UpdateFeaturesResponse createWithErrors(ApiError 
topLevelError, Map<String, ApiError> updateErrors, int throttleTimeMs) {
+    public static UpdateFeaturesResponse createWithErrors(short version, 
ApiError topLevelError, Map<String, ApiError> updateErrors, int throttleTimeMs) 
{
         final UpdatableFeatureResultCollection results = new 
UpdatableFeatureResultCollection();
-        for (final Map.Entry<String, ApiError> updateError : 
updateErrors.entrySet()) {
-            final String feature = updateError.getKey();
-            final ApiError error = updateError.getValue();
-            final UpdatableFeatureResult result = new UpdatableFeatureResult();
-            result.setFeature(feature)
-                .setErrorCode(error.error().code())
-                .setErrorMessage(error.message());
-            results.add(result);
+        Optional<Map.Entry<String, ApiError>> errorEntry = Optional.empty();
+        if (version > 1) {
+            Stream<Map.Entry<String, ApiError>> errorEntries = 
updateErrors.entrySet().stream().filter(entry ->

Review Comment:
   >  In the case where it would be called with a partial update, we would 
potentially want to catch the error?
   
   But the ZK response is hard coded for v1, right?



##########
clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java:
##########
@@ -82,17 +84,31 @@ public static UpdateFeaturesResponse parse(ByteBuffer 
buffer, short version) {
         return new UpdateFeaturesResponse(new UpdateFeaturesResponseData(new 
ByteBufferAccessor(buffer), version));
     }
 
-    public static UpdateFeaturesResponse createWithErrors(ApiError 
topLevelError, Map<String, ApiError> updateErrors, int throttleTimeMs) {
+    public static UpdateFeaturesResponse createWithErrors(short version, 
ApiError topLevelError, Map<String, ApiError> updateErrors, int throttleTimeMs) 
{
         final UpdatableFeatureResultCollection results = new 
UpdatableFeatureResultCollection();
-        for (final Map.Entry<String, ApiError> updateError : 
updateErrors.entrySet()) {
-            final String feature = updateError.getKey();
-            final ApiError error = updateError.getValue();
-            final UpdatableFeatureResult result = new UpdatableFeatureResult();
-            result.setFeature(feature)
-                .setErrorCode(error.error().code())
-                .setErrorMessage(error.message());
-            results.add(result);
+        Optional<Map.Entry<String, ApiError>> errorEntry = Optional.empty();
+        if (version > 1) {
+            Stream<Map.Entry<String, ApiError>> errorEntries = 
updateErrors.entrySet().stream().filter(entry ->
+                    !entry.getValue().error().equals(Errors.NONE));
+            errorEntry = errorEntries.findFirst();
         }
+
+        if (errorEntry.isPresent()) {
+            String errorFeatureName = errorEntry.get().getKey();

Review Comment:
   Hmm, we set `topLevelError` to topError in line 100. This means that if 
there is a feature level error, the top level error is not none, which is 
different from the current behavior in v1.



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