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


##########
clients/src/main/java/org/apache/kafka/clients/admin/DeleteStreamsGroupOffsetsResult.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.protocol.Errors;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * The result of the {@link Admin#deleteConsumerGroupOffsets(String, Set)} 
call.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DeleteStreamsGroupOffsetsResult extends 
DeleteConsumerGroupOffsetsResult {

Review Comment:
   I don't think we should use inheritance here. There is no subtyping 
relationship here really. Also, technically this is a change to the public API 
that needs to be changed in the KIP.
   
   I understand you want to avoid reimplementing the class. Have you considered 
using delegation? That is, the (package-protected) contructor for this class 
takes a DeleteConsumerGroupOffsetsResult and delegates calls to partitionResult 
and all to it?
   
   



##########
clients/src/main/java/org/apache/kafka/clients/admin/AlterStreamsGroupOffsetsResult.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.protocol.Errors;
+
+import java.util.Map;
+
+/**
+ * The result of the {@link AdminClient#alterConsumerGroupOffsets(String, 
Map)} call.

Review Comment:
   This is the result of alterStreamsGroupOffsets



##########
clients/src/main/java/org/apache/kafka/clients/admin/Admin.java:
##########
@@ -945,6 +945,29 @@ default ListConsumerGroupOffsetsResult 
listConsumerGroupOffsets(Map<String, List
         return listConsumerGroupOffsets(groupSpecs, new 
ListConsumerGroupOffsetsOptions());
     }
 
+    /**

Review Comment:
   For all the javadocs for methods in the `Admin`, I'd suggest we add a 
`@note` that this effectively does the same as the corresponding consumer group 
method.



##########
clients/src/main/java/org/apache/kafka/clients/admin/AlterStreamsGroupOffsetsOptions.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Map;
+
+/**
+ * Options for the {@link AdminClient#alterStreamsGroupOffsets(String, Map, 
AlterStreamsGroupOffsetsOptions)} call.
+ *
+ * The API of this class is evolving, see {@link AdminClient} for details.
+ */
+@InterfaceStability.Evolving
+public class AlterStreamsGroupOffsetsOptions extends 
AlterConsumerGroupOffsetsOptions {

Review Comment:
   I would directly derive from 
`AbstractOptions<AlterStreamsGroupOffsetsOptions>` here. Similar for the other 
`option` classes.



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