You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/12/09 04:54:36 UTC

[GitHub] [kafka] chia7712 commented on a change in pull request #9714: MINOR: Remove connection id from Send and consolidate request/message utils

chia7712 commented on a change in pull request #9714:
URL: https://github.com/apache/kafka/pull/9714#discussion_r538994089



##########
File path: clients/src/main/java/org/apache/kafka/common/record/MultiRecordsSend.java
##########
@@ -46,8 +45,7 @@
      * Construct a MultiRecordsSend for the given destination from a queue of Send objects. The queue will be

Review comment:
       This comment need to be updated.

##########
File path: clients/src/main/java/org/apache/kafka/common/requests/RequestUtils.java
##########
@@ -76,36 +75,22 @@ public static boolean hasTransactionalRecords(ProduceRequest request) {
         return new AbstractMap.SimpleEntry<>(hasIdempotentRecords, hasTransactionalRecords);
     }
 
-    public static MessageSizeAccumulator size(
-        ObjectSerializationCache serializationCache,
-        Message header,
-        short headerVersion,
-        Message apiMessage,
-        short apiVersion
-    ) {
-        MessageSizeAccumulator messageSize = new MessageSizeAccumulator();
-        if (header != null)
-            header.addSize(messageSize, serializationCache, headerVersion);
-        apiMessage.addSize(messageSize, serializationCache, apiVersion);
-        return messageSize;
-    }
-
     public static ByteBuffer serialize(
         Message header,
         short headerVersion,
         Message apiMessage,
         short apiVersion
     ) {
-        ObjectSerializationCache serializationCache = new ObjectSerializationCache();
-        MessageSizeAccumulator messageSize = RequestUtils.size(serializationCache, header, headerVersion, apiMessage, apiVersion);
+        ObjectSerializationCache cache = new ObjectSerializationCache();
+
+        int headerSize = header.size(cache, headerVersion);
+        int messageSize = apiMessage.size(cache, apiVersion);
+        ByteBufferAccessor writable = new ByteBufferAccessor(ByteBuffer.allocate(4 + headerSize + messageSize));

Review comment:
       Why it need 4 byte if it exclude size prefix?

##########
File path: core/src/main/scala/kafka/network/SocketServer.scala
##########
@@ -1134,11 +1133,11 @@ private[kafka] class Processor(val id: Int,
         // Try unmuting the channel. If there was no quota violation and the channel has not been throttled,
         // it will be unmuted immediately. If the channel has been throttled, it will unmuted only if the throttling
         // delay has already passed by now.
-        handleChannelMuteEvent(send.destination, ChannelMuteEvent.RESPONSE_SENT)
-        tryUnmuteChannel(send.destination)
+        handleChannelMuteEvent(send.destinationId(), ChannelMuteEvent.RESPONSE_SENT)
+        tryUnmuteChannel(send.destinationId())

Review comment:
       How about keeping "parameterless"? fewer changes and more clear.

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/RequestTestUtils.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.common.requests;
+
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.ObjectSerializationCache;
+import org.apache.kafka.common.record.RecordBatch;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Function;
+
+public class RequestTestUtils {
+
+    public static ByteBuffer serializeRequestHeader(RequestHeader header) {
+        ObjectSerializationCache serializationCache = new ObjectSerializationCache();
+        ByteBuffer buffer = ByteBuffer.allocate(header.size(serializationCache));
+        header.write(buffer, serializationCache);

Review comment:
       ```RequestHeader#write``` has only 2 usages and both of them are in test scope. It should be fine to remove ```RequestHeader#write``` from production.

##########
File path: core/src/main/scala/kafka/network/SocketServer.scala
##########
@@ -964,7 +963,7 @@ private[kafka] class Processor(val id: Int,
     // removed from the Selector after discarding any pending staged receives.
     // `openOrClosingChannel` can be None if the selector closed the connection because it was idle for too long
     if (openOrClosingChannel(connectionId).isDefined) {
-      selector.send(responseSend)
+      selector.send(new NetworkSend(response.request.context.connectionId, responseSend))

Review comment:
       line#955 already has ```connectionId```.

##########
File path: clients/src/main/java/org/apache/kafka/common/network/Send.java
##########
@@ -24,11 +24,6 @@
  */
 public interface Send {

Review comment:
       please remove "destination" from the docs

##########
File path: clients/src/test/java/org/apache/kafka/common/requests/RequestTestUtils.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.common.requests;
+
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.ObjectSerializationCache;
+import org.apache.kafka.common.record.RecordBatch;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Function;
+
+public class RequestTestUtils {
+
+    public static ByteBuffer serializeRequestHeader(RequestHeader header) {
+        ObjectSerializationCache serializationCache = new ObjectSerializationCache();
+        ByteBuffer buffer = ByteBuffer.allocate(header.size(serializationCache));
+        header.write(buffer, serializationCache);
+        buffer.flip();
+        return buffer;
+    }
+
+    public static ByteBuffer serializeRequestWithHeader(RequestHeader header, AbstractRequest request) {
+        return RequestUtils.serialize(header.data(), header.headerVersion(), request.data(), request.version());
+    }
+
+    public static ByteBuffer serializeResponseWithHeader(AbstractResponse response, short version, int correlationId) {
+        return response.serializeWithHeader(new ResponseHeader(correlationId,
+            response.apiKey().responseHeaderVersion(version)), version);
+    }
+
+    public static MetadataResponse metadataResponse(Collection<Node> brokers,
+                                                    String clusterId, int controllerId,
+                                                    List<MetadataResponse.TopicMetadata> topicMetadataList) {
+        return metadataResponse(MetadataResponse.DEFAULT_THROTTLE_TIME, brokers, clusterId, controllerId,
+                topicMetadataList, MetadataResponse.AUTHORIZED_OPERATIONS_OMITTED);
+    }
+
+    public static MetadataResponse metadataResponse(int throttleTimeMs, Collection<Node> brokers,
+                                                    String clusterId, int controllerId,
+                                                    List<MetadataResponse.TopicMetadata> topicMetadatas,
+                                                    int clusterAuthorizedOperations) {
+        List<MetadataResponseData.MetadataResponseTopic> topics = new ArrayList<>();
+        topicMetadatas.forEach(topicMetadata -> {
+            MetadataResponseData.MetadataResponseTopic metadataResponseTopic = new MetadataResponseData.MetadataResponseTopic();
+            metadataResponseTopic
+                    .setErrorCode(topicMetadata.error().code())
+                    .setName(topicMetadata.topic())
+                    .setIsInternal(topicMetadata.isInternal())
+                    .setTopicAuthorizedOperations(topicMetadata.authorizedOperations());
+
+            for (MetadataResponse.PartitionMetadata partitionMetadata : topicMetadata.partitionMetadata()) {
+                metadataResponseTopic.partitions().add(new MetadataResponseData.MetadataResponsePartition()
+                        .setErrorCode(partitionMetadata.error.code())
+                        .setPartitionIndex(partitionMetadata.partition())
+                        .setLeaderId(partitionMetadata.leaderId.orElse(MetadataResponse.NO_LEADER_ID))
+                        .setLeaderEpoch(partitionMetadata.leaderEpoch.orElse(RecordBatch.NO_PARTITION_LEADER_EPOCH))
+                        .setReplicaNodes(partitionMetadata.replicaIds)
+                        .setIsrNodes(partitionMetadata.inSyncReplicaIds)
+                        .setOfflineReplicas(partitionMetadata.offlineReplicaIds));
+            }
+            topics.add(metadataResponseTopic);
+        });
+        return MetadataResponse.prepareResponse(true, throttleTimeMs, brokers, clusterId, controllerId,
+                topics, clusterAuthorizedOperations); }
+
+    public static MetadataResponse metadataUpdateWith(final int numNodes,
+                                                      final Map<String, Integer> topicPartitionCounts) {
+        return metadataUpdateWith("kafka-cluster", numNodes, topicPartitionCounts);
+    }
+
+    public static MetadataResponse metadataUpdateWith(final int numNodes,
+                                                      final Map<String, Integer> topicPartitionCounts,
+                                                      final Function<TopicPartition, Integer> epochSupplier) {
+        return metadataUpdateWith("kafka-cluster", numNodes, Collections.emptyMap(),
+                topicPartitionCounts, epochSupplier, MetadataResponse.PartitionMetadata::new, ApiKeys.METADATA.latestVersion());
+    }
+
+    public static MetadataResponse metadataUpdateWith(final String clusterId,
+                                                      final int numNodes,
+                                                      final Map<String, Integer> topicPartitionCounts) {
+        return metadataUpdateWith(clusterId, numNodes, Collections.emptyMap(),
+                topicPartitionCounts, tp -> null, MetadataResponse.PartitionMetadata::new, ApiKeys.METADATA.latestVersion());
+    }
+
+    public static MetadataResponse metadataUpdateWith(final String clusterId,
+                                                      final int numNodes,
+                                                      final Map<String, Errors> topicErrors,
+                                                      final Map<String, Integer> topicPartitionCounts) {
+        return metadataUpdateWith(clusterId, numNodes, topicErrors,
+                topicPartitionCounts, tp -> null, MetadataResponse.PartitionMetadata::new, ApiKeys.METADATA.latestVersion());
+    }
+
+    public static MetadataResponse metadataUpdateWith(final String clusterId,
+                                                      final int numNodes,
+                                                      final Map<String, Errors> topicErrors,
+                                                      final Map<String, Integer> topicPartitionCounts,
+                                                      final short responseVersion) {

Review comment:
       unused argument




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org