diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 8ebe7ed..a016269 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -12,7 +12,6 @@ */ package org.apache.kafka.clients.producer.internals; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; @@ -277,16 +276,13 @@ public class Sender implements Runnable { * Create a produce request from the given record batches */ private ClientRequest produceRequest(long now, int destination, short acks, int timeout, List batches) { - Map produceRecordsByPartition = new HashMap(batches.size()); + ProduceRequest request = new ProduceRequest(acks, timeout); Map recordsByPartition = new HashMap(batches.size()); for (RecordBatch batch : batches) { - TopicPartition tp = batch.topicPartition; - ByteBuffer recordsBuffer = batch.records.buffer(); - recordsBuffer.flip(); - produceRecordsByPartition.put(tp, recordsBuffer); - recordsByPartition.put(tp, batch); + batch.records.buffer().flip(); + request.add(batch.topicPartition, batch.records); + recordsByPartition.put(batch.topicPartition, batch); } - ProduceRequest request = new ProduceRequest(acks, timeout, produceRecordsByPartition); RequestSend send = new RequestSend(destination, this.client.nextRequestHeader(ApiKeys.PRODUCE), request.toStruct()); return new ClientRequest(now, acks != 0, send, recordsByPartition); } diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index d3299b9..c62707a 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -15,7 +15,12 @@ package org.apache.kafka.common; import org.apache.kafka.common.utils.Utils; import java.net.InetSocketAddress; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; /** * A representation of a subset of the nodes, topics, and partitions in the Kafka cluster. @@ -138,14 +143,6 @@ public final class Cluster { return this.partitionsByNode.get(nodeId); } - /** - * Get all topics. - * @return a set of all topics - */ - public Set topics() { - return this.partitionsByTopic.keySet(); - } - @Override public String toString() { return "Cluster(nodes = " + this.nodes + ", partitions = " + this.partitionsByTopicPartition.values() + ")"; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 109fc96..6fe7573 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -30,11 +30,8 @@ public enum ApiKeys { METADATA(3, "metadata"), LEADER_AND_ISR(4, "leader_and_isr"), STOP_REPLICA(5, "stop_replica"), - OFFSET_COMMIT(8, "offset_commit"), - OFFSET_FETCH(9, "offset_fetch"), - CONSUMER_METADATA(10, "consumer_metadata"), - JOIN_GROUP(11, "join_group"), - HEARTBEAT(12, "heartbeat"); + OFFSET_COMMIT(6, "offset_commit"), + OFFSET_FETCH(7, "offset_fetch"); private static ApiKeys[] codeToType; public static int MAX_API_KEY = -1; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 7517b87..044b030 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -104,264 +104,6 @@ public class Protocol { public static Schema[] PRODUCE_REQUEST = new Schema[] { PRODUCE_REQUEST_V0 }; public static Schema[] PRODUCE_RESPONSE = new Schema[] { PRODUCE_RESPONSE_V0 }; - /* Offset commit api */ - public static Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("offset", - INT64, - "Message offset to be committed."), - new Field("timestamp", - INT64, - "Timestamp of the commit"), - new Field("metadata", - STRING, - "Any associated metadata the client wants to keep.")); - - public static Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(new Field("topic", - STRING, - "Topic to commit."), - new Field("partitions", - new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V0), - "Partitions to commit offsets.")); - - public static Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id", - STRING, - "The consumer group id."), - new Field("topics", - new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), - "Topics to commit offsets.")); - - public static Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(new Field("group_id", - STRING, - "The consumer group id."), - new Field("group_generation_id", - INT32, - "The generation of the consumer group."), - new Field("consumer_id", - STRING, - "The consumer id assigned by the group coordinator."), - new Field("topics", - new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), - "Topics to commit offsets.")); - - public static Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("error_code", - INT16)); - - public static Schema OFFSET_COMMIT_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), - new Field("partition_responses", - new ArrayOf(OFFSET_COMMIT_RESPONSE_PARTITION_V0))); - - public static Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0))); - - public static Schema[] OFFSET_COMMIT_REQUEST = new Schema[] { OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1 }; - /* The response types for both V0 and V1 of OFFSET_COMMIT_REQUEST are the same. */ - public static Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] { OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0}; - - /* Offset fetch api */ - public static Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id.")); - - public static Schema OFFSET_FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic", - STRING, - "Topic to fetch offset."), - new Field("partitions", - new ArrayOf(OFFSET_FETCH_REQUEST_PARTITION_V0), - "Partitions to fetch offsets.")); - - public static Schema OFFSET_FETCH_REQUEST_V0 = new Schema(new Field("group_id", - STRING, - "The consumer group id."), - new Field("topics", - new ArrayOf(OFFSET_FETCH_REQUEST_TOPIC_V0), - "Topics to fetch offsets.")); - - public static Schema OFFSET_FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("offset", - INT64, - "Last committed message offset."), - new Field("metadata", - STRING, - "Any associated metadata the client wants to keep."), - new Field("error_code", - INT16)); - - public static Schema OFFSET_FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), - new Field("partition_responses", - new ArrayOf(OFFSET_FETCH_RESPONSE_PARTITION_V0))); - - public static Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0))); - - public static Schema[] OFFSET_FETCH_REQUEST = new Schema[] { OFFSET_FETCH_REQUEST_V0 }; - public static Schema[] OFFSET_FETCH_RESPONSE = new Schema[] { OFFSET_FETCH_RESPONSE_V0 }; - - /* List offset api */ - public static Schema LIST_OFFSET_REQUEST_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("timestamp", - INT64, - "Timestamp."), - new Field("max_num_offsets", - INT32, - "Maximum offsets to return.")); - - public static Schema LIST_OFFSET_REQUEST_TOPIC_V0 = new Schema(new Field("topic", - STRING, - "Topic to list offset."), - new Field("partitions", - new ArrayOf(LIST_OFFSET_REQUEST_PARTITION_V0), - "Partitions to list offset.")); - - public static Schema LIST_OFFSET_REQUEST_V0 = new Schema(new Field("replica_id", - INT32, - "Broker id of the follower. For normal consumers, use -1."), - new Field("topics", - new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V0), - "Topics to list offsets.")); - - public static Schema LIST_OFFSET_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("error_code", - INT16), - new Field("offsets", - new ArrayOf(INT64), - "A list of offsets.")); - - public static Schema LIST_OFFSET_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), - new Field("partition_responses", - new ArrayOf(LIST_OFFSET_RESPONSE_PARTITION_V0))); - - public static Schema LIST_OFFSET_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V0))); - - public static Schema[] LIST_OFFSET_REQUEST = new Schema[] { LIST_OFFSET_REQUEST_V0 }; - public static Schema[] LIST_OFFSET_RESPONSE = new Schema[] { LIST_OFFSET_RESPONSE_V0 }; - - /* Fetch api */ - public static Schema FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("fetch_offset", - INT64, - "Message offset."), - new Field("max_bytes", - INT32, - "Maximum bytes to fetch.")); - - public static Schema FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic", - STRING, - "Topic to fetch."), - new Field("partitions", - new ArrayOf(FETCH_REQUEST_PARTITION_V0), - "Partitions to fetch.")); - - public static Schema FETCH_REQUEST_V0 = new Schema(new Field("replica_id", - INT32, - "Broker id of the follower. For normal consumers, use -1."), - new Field("max_wait_time", - INT32, - "Maximum time in ms to wait for the response."), - new Field("min_bytes", - INT32, - "Minimum bytes to accumulate in the response."), - new Field("topics", - new ArrayOf(FETCH_REQUEST_TOPIC_V0), - "Topics to fetch.")); - - public static Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("error_code", - INT16), - new Field("high_watermark", - INT64, - "Last committed offset."), - new Field("record_set", BYTES)); - - public static Schema FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), - new Field("partition_responses", - new ArrayOf(FETCH_RESPONSE_PARTITION_V0))); - - public static Schema FETCH_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(FETCH_RESPONSE_TOPIC_V0))); - - public static Schema[] FETCH_REQUEST = new Schema[] { FETCH_REQUEST_V0 }; - public static Schema[] FETCH_RESPONSE = new Schema[] { FETCH_RESPONSE_V0 }; - - /* Consumer metadata api */ - public static Schema CONSUMER_METADATA_REQUEST_V0 = new Schema(new Field("group_id", - STRING, - "The consumer group id.")); - - public static Schema CONSUMER_METADATA_RESPONSE_V0 = new Schema(new Field("error_code", - INT16), - new Field("coordinator", - BROKER, - "Host and port information for the coordinator for a consumer group.")); - - public static Schema[] CONSUMER_METADATA_REQUEST = new Schema[] { CONSUMER_METADATA_REQUEST_V0 }; - public static Schema[] CONSUMER_METADATA_RESPONSE = new Schema[] { CONSUMER_METADATA_RESPONSE_V0 }; - - /* Join group api */ - public static Schema JOIN_GROUP_REQUEST_V0 = new Schema(new Field("group_id", - STRING, - "The consumer group id."), - new Field("session_timeout", - INT32, - "The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms."), - new Field("topics", - new ArrayOf(STRING), - "An array of topics to subscribe to."), - new Field("consumer_id", - STRING, - "The assigned consumer id or an empty string for a new consumer."), - new Field("partition_assignment_strategy", - STRING, - "The strategy for the coordinator to assign partitions.")); - - public static Schema JOIN_GROUP_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), - new Field("partitions", new ArrayOf(INT32))); - public static Schema JOIN_GROUP_RESPONSE_V0 = new Schema(new Field("error_code", - INT16), - new Field("group_generation_id", - INT32, - "The generation of the consumer group."), - new Field("consumer_id", - STRING, - "The consumer id assigned by the group coordinator."), - new Field("assigned_partitions", - new ArrayOf(JOIN_GROUP_RESPONSE_TOPIC_V0))); - - public static Schema[] JOIN_GROUP_REQUEST = new Schema[] { JOIN_GROUP_REQUEST_V0 }; - public static Schema[] JOIN_GROUP_RESPONSE = new Schema[] { JOIN_GROUP_RESPONSE_V0 }; - - /* Heartbeat api */ - public static Schema HEARTBEAT_REQUEST_V0 = new Schema(new Field("group_id", - STRING, - "The consumer group id."), - new Field("group_generation_id", - INT32, - "The generation of the consumer group."), - new Field("consumer_id", - STRING, - "The consumer id assigned by the group coordinator.")); - - public static Schema HEARTBEAT_RESPONSE_V0 = new Schema(new Field("error_code", - INT16)); - - public static Schema[] HEARTBEAT_REQUEST = new Schema[] {HEARTBEAT_REQUEST_V0}; - public static Schema[] HEARTBEAT_RESPONSE = new Schema[] {HEARTBEAT_RESPONSE_V0}; - /* an array of all requests and responses with all schema versions */ public static Schema[][] REQUESTS = new Schema[ApiKeys.MAX_API_KEY + 1][]; public static Schema[][] RESPONSES = new Schema[ApiKeys.MAX_API_KEY + 1][]; @@ -371,28 +113,22 @@ public class Protocol { static { REQUESTS[ApiKeys.PRODUCE.id] = PRODUCE_REQUEST; - REQUESTS[ApiKeys.FETCH.id] = FETCH_REQUEST; - REQUESTS[ApiKeys.LIST_OFFSETS.id] = LIST_OFFSET_REQUEST; + REQUESTS[ApiKeys.FETCH.id] = new Schema[] {}; + REQUESTS[ApiKeys.LIST_OFFSETS.id] = new Schema[] {}; REQUESTS[ApiKeys.METADATA.id] = METADATA_REQUEST; REQUESTS[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {}; REQUESTS[ApiKeys.STOP_REPLICA.id] = new Schema[] {}; - REQUESTS[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_REQUEST; - REQUESTS[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_REQUEST; - REQUESTS[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_REQUEST; - REQUESTS[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_REQUEST; - REQUESTS[ApiKeys.HEARTBEAT.id] = HEARTBEAT_REQUEST; + REQUESTS[ApiKeys.OFFSET_COMMIT.id] = new Schema[] {}; + REQUESTS[ApiKeys.OFFSET_FETCH.id] = new Schema[] {}; RESPONSES[ApiKeys.PRODUCE.id] = PRODUCE_RESPONSE; - RESPONSES[ApiKeys.FETCH.id] = FETCH_RESPONSE; - RESPONSES[ApiKeys.LIST_OFFSETS.id] = LIST_OFFSET_RESPONSE; + RESPONSES[ApiKeys.FETCH.id] = new Schema[] {}; + RESPONSES[ApiKeys.LIST_OFFSETS.id] = new Schema[] {}; RESPONSES[ApiKeys.METADATA.id] = METADATA_RESPONSE; RESPONSES[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {}; RESPONSES[ApiKeys.STOP_REPLICA.id] = new Schema[] {}; - RESPONSES[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_RESPONSE; - RESPONSES[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_RESPONSE; - RESPONSES[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_RESPONSE; - RESPONSES[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_RESPONSE; - RESPONSES[ApiKeys.HEARTBEAT.id] = HEARTBEAT_RESPONSE; + RESPONSES[ApiKeys.OFFSET_COMMIT.id] = new Schema[] {}; + RESPONSES[ApiKeys.OFFSET_FETCH.id] = new Schema[] {}; /* set the maximum version of each api */ for (ApiKeys api : ApiKeys.values()) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java index 444e69e..8cecba5 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java @@ -83,15 +83,6 @@ public class Struct { return getFieldOrDefault(field); } - /** - * Check if the struct contains a field. - * @param name - * @return - */ - public boolean hasField(String name) { - return schema.get(name) != null; - } - public Struct getStruct(Field field) { return (Struct) get(field); } @@ -116,22 +107,6 @@ public class Struct { return (Integer) get(name); } - public Long getLong(Field field) { - return (Long) get(field); - } - - public Long getLong(String name) { - return (Long) get(name); - } - - public ByteBuffer getBytes(Field field) { - return (ByteBuffer) get(field); - } - - public ByteBuffer getBytes(String name) { - return (ByteBuffer) get(name); - } - public Object[] getArray(Field field) { return (Object[]) get(field); } @@ -278,46 +253,4 @@ public class Struct { return b.toString(); } - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - for (int i = 0; i < this.values.length; i++) { - Field f = this.schema.get(i); - if (f.type() instanceof ArrayOf) { - Object[] arrayObject = (Object []) this.get(f); - for (Object arrayItem: arrayObject) - result = prime * result + arrayItem.hashCode(); - } else { - result = prime * result + this.get(f).hashCode(); - } - } - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - Struct other = (Struct) obj; - if (schema != other.schema) - return false; - for (int i = 0; i < this.values.length; i++) { - Field f = this.schema.get(i); - Boolean result; - if (f.type() instanceof ArrayOf) { - result = Arrays.equals((Object []) this.get(f), (Object []) other.get(f)); - } else { - result = this.get(f).equals(other.get(f)); - } - if (!result) - return false; - } - return true; - } - } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java deleted file mode 100644 index 37aff6c..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java +++ /dev/null @@ -1,66 +0,0 @@ -/** - * 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.protocol.types.Struct; - -import java.nio.ByteBuffer; - -public abstract class AbstractRequestResponse { - protected final Struct struct; - - - public AbstractRequestResponse(Struct struct) { - this.struct = struct; - } - - public Struct toStruct() { - return struct; - } - - /** - * Get the serialized size of this object - */ - public int sizeOf() { - return struct.sizeOf(); - } - - /** - * Write this object to a buffer - */ - public void writeTo(ByteBuffer buffer) { - struct.writeTo(buffer); - } - - @Override - public String toString() { - return struct.toString(); - } - - @Override - public int hashCode() { - return struct.hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - AbstractRequestResponse other = (AbstractRequestResponse) obj; - return struct.equals(other.struct); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java deleted file mode 100644 index 99b52c2..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java +++ /dev/null @@ -1,47 +0,0 @@ -/** - * 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.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; - -import java.nio.ByteBuffer; - -public class ConsumerMetadataRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.CONSUMER_METADATA.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - - private final String groupId; - - public ConsumerMetadataRequest(String groupId) { - super(new Struct(curSchema)); - - struct.set(GROUP_ID_KEY_NAME, groupId); - this.groupId = groupId; - } - - public ConsumerMetadataRequest(Struct struct) { - super(struct); - groupId = struct.getString(GROUP_ID_KEY_NAME); - } - - public String groupId() { - return groupId; - } - - public static ConsumerMetadataRequest parse(ByteBuffer buffer) { - return new ConsumerMetadataRequest(((Struct) curSchema.read(buffer))); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java deleted file mode 100644 index 8b8f591..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java +++ /dev/null @@ -1,69 +0,0 @@ -/** - * 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.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; - -import java.nio.ByteBuffer; - -public class ConsumerMetadataResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.CONSUMER_METADATA.id); - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String COORDINATOR_KEY_NAME = "coordinator"; - - // coordinator level field names - private static String NODE_ID_KEY_NAME = "node_id"; - private static String HOST_KEY_NAME = "host"; - private static String PORT_KEY_NAME = "port"; - - private final short errorCode; - private final Node node; - - public ConsumerMetadataResponse(short errorCode, Node node) { - super(new Struct(curSchema)); - struct.set(ERROR_CODE_KEY_NAME, errorCode); - Struct coordinator = struct.instance(COORDINATOR_KEY_NAME); - coordinator.set(NODE_ID_KEY_NAME, node.id()); - coordinator.set(HOST_KEY_NAME, node.host()); - coordinator.set(PORT_KEY_NAME, node.port()); - struct.set(COORDINATOR_KEY_NAME, coordinator); - this.errorCode = errorCode; - this.node = node; - } - - public ConsumerMetadataResponse(Struct struct) { - super(struct); - errorCode = struct.getShort(ERROR_CODE_KEY_NAME); - Struct broker = (Struct) struct.get(COORDINATOR_KEY_NAME); - int nodeId = broker.getInt(NODE_ID_KEY_NAME); - String host = broker.getString(HOST_KEY_NAME); - int port = broker.getInt(PORT_KEY_NAME); - node = new Node(nodeId, host, port); - } - - public short errorCode() { - return errorCode; - } - - public Node node() { - return node; - } - - public static ConsumerMetadataResponse parse(ByteBuffer buffer) { - return new ConsumerMetadataResponse(((Struct) curSchema.read(buffer))); - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java deleted file mode 100644 index 2fc471f..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ /dev/null @@ -1,132 +0,0 @@ -/** - * 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.TopicPartition; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.utils.CollectionUtils; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class FetchRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.FETCH.id); - private static String REPLICA_ID_KEY_NAME = "replica_id"; - private static String MAX_WAIT_KEY_NAME = "max_wait_time"; - private static String MIN_BYTES_KEY_NAME = "min_bytes"; - private static String TOPICS_KEY_NAME = "topics"; - - // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; - - // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String FETCH_OFFSET_KEY_NAME = "fetch_offset"; - private static String MAX_BYTES_KEY_NAME = "max_bytes"; - - private final int replicaId; - private final int maxWait; - private final int minBytes; - private final Map fetchData; - - public static final class PartitionData { - public final long offset; - public final int maxBytes; - - public PartitionData(long offset, int maxBytes) { - this.offset = offset; - this.maxBytes = maxBytes; - } - } - - public FetchRequest(int replicaId, int maxWait, int minBytes, Map fetchData) { - super(new Struct(curSchema)); - Map> topicsData = CollectionUtils.groupDataByTopic(fetchData); - - struct.set(REPLICA_ID_KEY_NAME, replicaId); - struct.set(MAX_WAIT_KEY_NAME, maxWait); - struct.set(MIN_BYTES_KEY_NAME, minBytes); - List topicArray = new ArrayList(); - for (Map.Entry> topicEntry: topicsData.entrySet()) { - Struct topicData = struct.instance(TOPICS_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); - List partitionArray = new ArrayList(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { - PartitionData fetchPartitionData = partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(FETCH_OFFSET_KEY_NAME, fetchPartitionData.offset); - partitionData.set(MAX_BYTES_KEY_NAME, fetchPartitionData.maxBytes); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(TOPICS_KEY_NAME, topicArray.toArray()); - this.replicaId = replicaId; - this.maxWait = maxWait; - this.minBytes = minBytes; - this.fetchData = fetchData; - } - - public FetchRequest(Struct struct) { - super(struct); - replicaId = struct.getInt(REPLICA_ID_KEY_NAME); - maxWait = struct.getInt(MAX_WAIT_KEY_NAME); - minBytes = struct.getInt(MIN_BYTES_KEY_NAME); - fetchData = new HashMap(); - for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) { - Struct topicResponse = (Struct) topicResponseObj; - String topic = topicResponse.getString(TOPIC_KEY_NAME); - for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { - Struct partitionResponse = (Struct) partitionResponseObj; - int partition = partitionResponse.getInt(PARTITION_KEY_NAME); - long offset = partitionResponse.getLong(FETCH_OFFSET_KEY_NAME); - int maxBytes = partitionResponse.getInt(MAX_BYTES_KEY_NAME); - PartitionData partitionData = new PartitionData(offset, maxBytes); - fetchData.put(new TopicPartition(topic, partition), partitionData); - } - } - } - - public int replicaId() { - return replicaId; - } - - public int maxWait() { - return maxWait; - } - - public int minBytes() { - return minBytes; - } - - public Map fetchData() { - return fetchData; - } - - public static FetchRequest parse(ByteBuffer buffer) { - return new FetchRequest(((Struct) curSchema.read(buffer))); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java deleted file mode 100644 index f719010..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ /dev/null @@ -1,110 +0,0 @@ -/** - * 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.TopicPartition; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.utils.CollectionUtils; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class FetchResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.FETCH.id); - private static String RESPONSES_KEY_NAME = "responses"; - - // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; - - // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String HIGH_WATERMARK_KEY_NAME = "high_watermark"; - private static String RECORD_SET_KEY_NAME = "record_set"; - - private final Map responseData; - - public static final class PartitionData { - public final short errorCode; - public final long highWatermark; - public final ByteBuffer recordSet; - - public PartitionData(short errorCode, long highWatermark, ByteBuffer recordSet) { - this.errorCode = errorCode; - this.highWatermark = highWatermark; - this.recordSet = recordSet; - } - } - - public FetchResponse(Map responseData) { - super(new Struct(curSchema)); - Map> topicsData = CollectionUtils.groupDataByTopic(responseData); - - List topicArray = new ArrayList(); - for (Map.Entry> topicEntry: topicsData.entrySet()) { - Struct topicData = struct.instance(RESPONSES_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); - List partitionArray = new ArrayList(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { - PartitionData fetchPartitionData = partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.errorCode); - partitionData.set(HIGH_WATERMARK_KEY_NAME, fetchPartitionData.highWatermark); - partitionData.set(RECORD_SET_KEY_NAME, fetchPartitionData.recordSet); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); - this.responseData = responseData; - } - - public FetchResponse(Struct struct) { - super(struct); - responseData = new HashMap(); - for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { - Struct topicResponse = (Struct) topicResponseObj; - String topic = topicResponse.getString(TOPIC_KEY_NAME); - for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { - Struct partitionResponse = (Struct) partitionResponseObj; - int partition = partitionResponse.getInt(PARTITION_KEY_NAME); - short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME); - long highWatermark = partitionResponse.getLong(HIGH_WATERMARK_KEY_NAME); - ByteBuffer recordSet = partitionResponse.getBytes(RECORD_SET_KEY_NAME); - PartitionData partitionData = new PartitionData(errorCode, highWatermark, recordSet); - responseData.put(new TopicPartition(topic, partition), partitionData); - } - } - } - - public Map responseData() { - return responseData; - } - - public static FetchResponse parse(ByteBuffer buffer) { - return new FetchResponse(((Struct) curSchema.read(buffer))); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java deleted file mode 100644 index 9512db2..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java +++ /dev/null @@ -1,64 +0,0 @@ -/** - * 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.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; - -import java.nio.ByteBuffer; - -public class HeartbeatRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.HEARTBEAT.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String GROUP_GENERATION_ID_KEY_NAME = "group_generation_id"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; - - private final String groupId; - private final int groupGenerationId; - private final String consumerId; - - public HeartbeatRequest(String groupId, int groupGenerationId, String consumerId) { - super(new Struct(curSchema)); - struct.set(GROUP_ID_KEY_NAME, groupId); - struct.set(GROUP_GENERATION_ID_KEY_NAME, groupGenerationId); - struct.set(CONSUMER_ID_KEY_NAME, consumerId); - this.groupId = groupId; - this.groupGenerationId = groupGenerationId; - this.consumerId = consumerId; - } - - public HeartbeatRequest(Struct struct) { - super(struct); - groupId = struct.getString(GROUP_ID_KEY_NAME); - groupGenerationId = struct.getInt(GROUP_GENERATION_ID_KEY_NAME); - consumerId = struct.getString(CONSUMER_ID_KEY_NAME); - } - - public String groupId() { - return groupId; - } - - public int groupGenerationId() { - return groupGenerationId; - } - - public String consumerId() { - return consumerId; - } - - public static HeartbeatRequest parse(ByteBuffer buffer) { - return new HeartbeatRequest(((Struct) curSchema.read(buffer))); - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java deleted file mode 100644 index 8997ffc..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java +++ /dev/null @@ -1,45 +0,0 @@ -/** - * 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.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; - -import java.nio.ByteBuffer; - -public class HeartbeatResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id); - private static String ERROR_CODE_KEY_NAME = "error_code"; - - private final short errorCode; - public HeartbeatResponse(short errorCode) { - super(new Struct(curSchema)); - struct.set(ERROR_CODE_KEY_NAME, errorCode); - this.errorCode = errorCode; - } - - public HeartbeatResponse(Struct struct) { - super(struct); - errorCode = struct.getShort(ERROR_CODE_KEY_NAME); - } - - public short errorCode() { - return errorCode; - } - - public static HeartbeatResponse parse(ByteBuffer buffer) { - return new HeartbeatResponse(((Struct) curSchema.read(buffer))); - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java deleted file mode 100644 index d6e91f3..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ /dev/null @@ -1,87 +0,0 @@ -/** - * 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.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.List; - -public class JoinGroupRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.JOIN_GROUP.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String SESSION_TIMEOUT_KEY_NAME = "session_timeout"; - private static String TOPICS_KEY_NAME = "topics"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; - private static String STRATEGY_KEY_NAME = "partition_assignment_strategy"; - - private final String groupId; - private final int sessionTimeout; - private final List topics; - private final String consumerId; - private final String strategy; - - public JoinGroupRequest(String groupId, int sessionTimeout, List topics, String consumerId, String strategy) { - super(new Struct(curSchema)); - struct.set(GROUP_ID_KEY_NAME, groupId); - struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout); - struct.set(TOPICS_KEY_NAME, topics.toArray()); - struct.set(CONSUMER_ID_KEY_NAME, consumerId); - struct.set(STRATEGY_KEY_NAME, strategy); - this.groupId = groupId; - this.sessionTimeout = sessionTimeout; - this.topics = topics; - this.consumerId = consumerId; - this.strategy = strategy; - } - - public JoinGroupRequest(Struct struct) { - super(struct); - groupId = struct.getString(GROUP_ID_KEY_NAME); - sessionTimeout = struct.getInt(SESSION_TIMEOUT_KEY_NAME); - Object[] topicsArray = struct.getArray(TOPICS_KEY_NAME); - topics = new ArrayList(); - for (Object topic: topicsArray) - topics.add((String) topic); - consumerId = struct.getString(CONSUMER_ID_KEY_NAME); - strategy = struct.getString(STRATEGY_KEY_NAME); - } - - public String groupId() { - return groupId; - } - - public int sessionTimeout() { - return sessionTimeout; - } - - public List topics() { - return topics; - } - - public String consumerId() { - return consumerId; - } - - public String strategy() { - return strategy; - } - - public static JoinGroupRequest parse(ByteBuffer buffer) { - return new JoinGroupRequest(((Struct) curSchema.read(buffer))); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java deleted file mode 100644 index efe8979..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java +++ /dev/null @@ -1,102 +0,0 @@ -/** - * 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.TopicPartition; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.utils.CollectionUtils; - -import java.nio.ByteBuffer; -import java.util.*; - -public class JoinGroupResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id); - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String GENERATION_ID_KEY_NAME = "group_generation_id"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; - private static String ASSIGNED_PARTITIONS_KEY_NAME = "assigned_partitions"; - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; - - public static int UNKNOWN_GENERATION_ID = -1; - public static String UNKNOWN_CONSUMER_ID = ""; - - private final short errorCode; - private final int generationId; - private final String consumerId; - private final List assignedPartitions; - - public JoinGroupResponse(short errorCode, int generationId, String consumerId, List assignedPartitions) { - super(new Struct(curSchema)); - - Map> partitionsByTopic = CollectionUtils.groupDataByTopic(assignedPartitions); - - struct.set(ERROR_CODE_KEY_NAME, errorCode); - struct.set(GENERATION_ID_KEY_NAME, generationId); - struct.set(CONSUMER_ID_KEY_NAME, consumerId); - List topicArray = new ArrayList(); - for (Map.Entry> entries: partitionsByTopic.entrySet()) { - Struct topicData = struct.instance(ASSIGNED_PARTITIONS_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, entries.getKey()); - topicData.set(PARTITIONS_KEY_NAME, entries.getValue().toArray()); - topicArray.add(topicData); - } - struct.set(ASSIGNED_PARTITIONS_KEY_NAME, topicArray.toArray()); - - this.errorCode = errorCode; - this.generationId = generationId; - this.consumerId = consumerId; - this.assignedPartitions = assignedPartitions; - } - - public JoinGroupResponse(short errorCode) { - this(errorCode, UNKNOWN_GENERATION_ID, UNKNOWN_CONSUMER_ID, Collections.emptyList()); - } - - public JoinGroupResponse(Struct struct) { - super(struct); - assignedPartitions = new ArrayList(); - for (Object topicDataObj : struct.getArray(ASSIGNED_PARTITIONS_KEY_NAME)) { - Struct topicData = (Struct) topicDataObj; - String topic = topicData.getString(TOPIC_KEY_NAME); - for (Object partitionObj : topicData.getArray(PARTITIONS_KEY_NAME)) - assignedPartitions.add(new TopicPartition(topic, (Integer) partitionObj)); - } - errorCode = struct.getShort(ERROR_CODE_KEY_NAME); - generationId = struct.getInt(GENERATION_ID_KEY_NAME); - consumerId = struct.getString(CONSUMER_ID_KEY_NAME); - } - - public short errorCode() { - return errorCode; - } - - public int generationId() { - return generationId; - } - - public String consumerId() { - return consumerId; - } - - public List assignedPartitions() { - return assignedPartitions; - } - - public static JoinGroupResponse parse(ByteBuffer buffer) { - return new JoinGroupResponse(((Struct) curSchema.read(buffer))); - } -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java deleted file mode 100644 index 99364c1..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java +++ /dev/null @@ -1,114 +0,0 @@ -/** - * 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.TopicPartition; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.utils.CollectionUtils; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class ListOffsetRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.LIST_OFFSETS.id); - private static String REPLICA_ID_KEY_NAME = "replica_id"; - private static String TOPICS_KEY_NAME = "topics"; - - // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; - - // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String TIMESTAMP_KEY_NAME = "timestamp"; - private static String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets"; - - private final int replicaId; - private final Map offsetData; - - public static final class PartitionData { - public final long timestamp; - public final int maxNumOffsets; - - public PartitionData(long timestamp, int maxNumOffsets) { - this.timestamp = timestamp; - this.maxNumOffsets = maxNumOffsets; - } - } - - public ListOffsetRequest(int replicaId, Map offsetData) { - super(new Struct(curSchema)); - Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); - - struct.set(REPLICA_ID_KEY_NAME, replicaId); - List topicArray = new ArrayList(); - for (Map.Entry> topicEntry: topicsData.entrySet()) { - Struct topicData = struct.instance(TOPICS_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); - List partitionArray = new ArrayList(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { - PartitionData offsetPartitionData = partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp); - partitionData.set(MAX_NUM_OFFSETS_KEY_NAME, offsetPartitionData.maxNumOffsets); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(TOPICS_KEY_NAME, topicArray.toArray()); - this.replicaId = replicaId; - this.offsetData = offsetData; - } - - public ListOffsetRequest(Struct struct) { - super(struct); - replicaId = struct.getInt(REPLICA_ID_KEY_NAME); - offsetData = new HashMap(); - for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) { - Struct topicResponse = (Struct) topicResponseObj; - String topic = topicResponse.getString(TOPIC_KEY_NAME); - for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { - Struct partitionResponse = (Struct) partitionResponseObj; - int partition = partitionResponse.getInt(PARTITION_KEY_NAME); - long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME); - int maxNumOffsets = partitionResponse.getInt(MAX_NUM_OFFSETS_KEY_NAME); - PartitionData partitionData = new PartitionData(timestamp, maxNumOffsets); - offsetData.put(new TopicPartition(topic, partition), partitionData); - } - } - } - - public int replicaId() { - return replicaId; - } - - public Map offsetData() { - return offsetData; - } - - public static ListOffsetRequest parse(ByteBuffer buffer) { - return new ListOffsetRequest(((Struct) curSchema.read(buffer))); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java deleted file mode 100644 index ac23971..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java +++ /dev/null @@ -1,108 +0,0 @@ -/** - * 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.TopicPartition; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.utils.CollectionUtils; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class ListOffsetResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.LIST_OFFSETS.id); - private static String RESPONSES_KEY_NAME = "responses"; - - // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; - - // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String OFFSETS_KEY_NAME = "offsets"; - - private final Map responseData; - - public static final class PartitionData { - public final short errorCode; - public final List offsets; - - public PartitionData(short errorCode, List offsets) { - this.errorCode = errorCode; - this.offsets = offsets; - } - } - - public ListOffsetResponse(Map responseData) { - super(new Struct(curSchema)); - Map> topicsData = CollectionUtils.groupDataByTopic(responseData); - - List topicArray = new ArrayList(); - for (Map.Entry> topicEntry: topicsData.entrySet()) { - Struct topicData = struct.instance(RESPONSES_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); - List partitionArray = new ArrayList(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { - PartitionData offsetPartitionData = partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(ERROR_CODE_KEY_NAME, offsetPartitionData.errorCode); - partitionData.set(OFFSETS_KEY_NAME, offsetPartitionData.offsets.toArray()); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); - this.responseData = responseData; - } - - public ListOffsetResponse(Struct struct) { - super(struct); - responseData = new HashMap(); - for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { - Struct topicResponse = (Struct) topicResponseObj; - String topic = topicResponse.getString(TOPIC_KEY_NAME); - for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { - Struct partitionResponse = (Struct) partitionResponseObj; - int partition = partitionResponse.getInt(PARTITION_KEY_NAME); - short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME); - Object[] offsets = partitionResponse.getArray(OFFSETS_KEY_NAME); - List offsetsList = new ArrayList(); - for (Object offset: offsets) - offsetsList.add((Long) offset); - PartitionData partitionData = new PartitionData(errorCode, offsetsList); - responseData.put(new TopicPartition(topic, partition), partitionData); - } - } - } - - public Map responseData() { - return responseData; - } - - public static ListOffsetResponse parse(ByteBuffer buffer) { - return new ListOffsetResponse(((Struct) curSchema.read(buffer))); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java index b22ca1d..f35bd87 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -12,41 +12,26 @@ */ package org.apache.kafka.common.requests; -import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.List; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; -public class MetadataRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); - private static String TOPICS_KEY_NAME = "topics"; +public class MetadataRequest { private final List topics; public MetadataRequest(List topics) { - super(new Struct(curSchema)); - struct.set(TOPICS_KEY_NAME, topics.toArray()); this.topics = topics; } - public MetadataRequest(Struct struct) { - super(struct); - Object[] topicArray = struct.getArray(TOPICS_KEY_NAME); - topics = new ArrayList(); - for (Object topicObj: topicArray) { - topics.add((String) topicObj); - } + public Struct toStruct() { + String[] ts = new String[topics.size()]; + topics.toArray(ts); + Struct body = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id)); + body.set("topics", topics.toArray()); + return body; } - public List topics() { - return topics; - } - - public static MetadataRequest parse(ByteBuffer buffer) { - return new MetadataRequest(((Struct) curSchema.read(buffer))); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index 7d90fce..2652c32 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -12,7 +12,6 @@ */ package org.apache.kafka.common.requests; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -21,112 +20,50 @@ import java.util.Map; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; -public class MetadataResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id); - private static String BROKERS_KEY_NAME = "brokers"; - private static String TOPIC_METATDATA_KEY_NAME = "topic_metadata"; - - // broker level field names - private static String NODE_ID_KEY_NAME = "node_id"; - private static String HOST_KEY_NAME = "host"; - private static String PORT_KEY_NAME = "port"; - - // topic level field names - private static String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code"; - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITION_METADATA_KEY_NAME = "partition_metadata"; - - // partition level field names - private static String PARTITION_ERROR_CODE_KEY_NAME = "partition_error_code"; - private static String PARTITION_KEY_NAME = "partition_id"; - private static String LEADER_KEY_NAME = "leader"; - private static String REPLICAS_KEY_NAME = "replicas"; - private static String ISR_KEY_NAME = "isr"; +public class MetadataResponse { private final Cluster cluster; private final Map errors; - public MetadataResponse(Cluster cluster) { - super(new Struct(curSchema)); - - List brokerArray = new ArrayList(); - for (Node node: cluster.nodes()) { - Struct broker = struct.instance(BROKERS_KEY_NAME); - broker.set(NODE_ID_KEY_NAME, node.id()); - broker.set(HOST_KEY_NAME, node.host()); - broker.set(PORT_KEY_NAME, node.port()); - brokerArray.add(broker); - } - struct.set(BROKERS_KEY_NAME, brokerArray.toArray()); - - List topicArray = new ArrayList(); - for (String topic: cluster.topics()) { - Struct topicData = struct.instance(TOPIC_METATDATA_KEY_NAME); - topicData.set(TOPIC_ERROR_CODE_KEY_NAME, (short)0); // no error - topicData.set(TOPIC_KEY_NAME, topic); - List partitionArray = new ArrayList(); - for (PartitionInfo fetchPartitionData : cluster.partitionsForTopic(topic)) { - Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME); - partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, (short)0); // no error - partitionData.set(PARTITION_KEY_NAME, fetchPartitionData.partition()); - partitionData.set(LEADER_KEY_NAME, fetchPartitionData.leader().id()); - ArrayList replicas = new ArrayList(); - for (Node node: fetchPartitionData.replicas()) - replicas.add(node.id()); - partitionData.set(REPLICAS_KEY_NAME, replicas.toArray()); - ArrayList isr = new ArrayList(); - for (Node node: fetchPartitionData.inSyncReplicas()) - isr.add(node.id()); - partitionData.set(ISR_KEY_NAME, isr.toArray()); - partitionArray.add(partitionData); - } - topicData.set(PARTITION_METADATA_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(TOPIC_METATDATA_KEY_NAME, topicArray.toArray()); - + public MetadataResponse(Cluster cluster, Map errors) { this.cluster = cluster; - this.errors = new HashMap(); + this.errors = errors; } public MetadataResponse(Struct struct) { - super(struct); Map errors = new HashMap(); Map brokers = new HashMap(); - Object[] brokerStructs = (Object[]) struct.get(BROKERS_KEY_NAME); + Object[] brokerStructs = (Object[]) struct.get("brokers"); for (int i = 0; i < brokerStructs.length; i++) { Struct broker = (Struct) brokerStructs[i]; - int nodeId = broker.getInt(NODE_ID_KEY_NAME); - String host = broker.getString(HOST_KEY_NAME); - int port = broker.getInt(PORT_KEY_NAME); + int nodeId = (Integer) broker.get("node_id"); + String host = (String) broker.get("host"); + int port = (Integer) broker.get("port"); brokers.put(nodeId, new Node(nodeId, host, port)); } List partitions = new ArrayList(); - Object[] topicInfos = (Object[]) struct.get(TOPIC_METATDATA_KEY_NAME); + Object[] topicInfos = (Object[]) struct.get("topic_metadata"); for (int i = 0; i < topicInfos.length; i++) { Struct topicInfo = (Struct) topicInfos[i]; - short topicError = topicInfo.getShort(TOPIC_ERROR_CODE_KEY_NAME); - String topic = topicInfo.getString(TOPIC_KEY_NAME); + short topicError = topicInfo.getShort("topic_error_code"); + String topic = topicInfo.getString("topic"); if (topicError == Errors.NONE.code()) { - Object[] partitionInfos = (Object[]) topicInfo.get(PARTITION_METADATA_KEY_NAME); + Object[] partitionInfos = (Object[]) topicInfo.get("partition_metadata"); for (int j = 0; j < partitionInfos.length; j++) { Struct partitionInfo = (Struct) partitionInfos[j]; - short partError = partitionInfo.getShort(PARTITION_ERROR_CODE_KEY_NAME); + short partError = partitionInfo.getShort("partition_error_code"); if (partError == Errors.NONE.code()) { - int partition = partitionInfo.getInt(PARTITION_KEY_NAME); - int leader = partitionInfo.getInt(LEADER_KEY_NAME); + int partition = partitionInfo.getInt("partition_id"); + int leader = partitionInfo.getInt("leader"); Node leaderNode = leader == -1 ? null : brokers.get(leader); - Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME); + Object[] replicas = (Object[]) partitionInfo.get("replicas"); Node[] replicaNodes = new Node[replicas.length]; for (int k = 0; k < replicas.length; k++) replicaNodes[k] = brokers.get(replicas[k]); - Object[] isr = (Object[]) partitionInfo.get(ISR_KEY_NAME); + Object[] isr = (Object[]) partitionInfo.get("isr"); Node[] isrNodes = new Node[isr.length]; for (int k = 0; k < isr.length; k++) isrNodes[k] = brokers.get(isr[k]); @@ -149,7 +86,4 @@ public class MetadataResponse extends AbstractRequestResponse { return this.cluster; } - public static MetadataResponse parse(ByteBuffer buffer) { - return new MetadataResponse(((Struct) curSchema.read(buffer))); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java deleted file mode 100644 index 3ee5cba..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ /dev/null @@ -1,180 +0,0 @@ -/** - * 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.TopicPartition; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.utils.CollectionUtils; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * This wrapper supports both v0 and v1 of OffsetCommitRequest. - */ -public class OffsetCommitRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_COMMIT.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String GENERATION_ID_KEY_NAME = "group_generation_id"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; - private static String TOPICS_KEY_NAME = "topics"; - - // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; - - // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String COMMIT_OFFSET_KEY_NAME = "offset"; - private static String TIMESTAMP_KEY_NAME = "timestamp"; - private static String METADATA_KEY_NAME = "metadata"; - - public static final int DEFAULT_GENERATION_ID = -1; - public static final String DEFAULT_CONSUMER_ID = ""; - - private final String groupId; - private final int generationId; - private final String consumerId; - private final Map offsetData; - - public static final class PartitionData { - public final long offset; - public final long timestamp; - public final String metadata; - - public PartitionData(long offset, long timestamp, String metadata) { - this.offset = offset; - this.timestamp = timestamp; - this.metadata = metadata; - } - } - - /** - * Constructor for version 0. - * @param groupId - * @param offsetData - */ - @Deprecated - public OffsetCommitRequest(String groupId, Map offsetData) { - super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 0))); - initCommonFields(groupId, offsetData); - this.groupId = groupId; - this.generationId = DEFAULT_GENERATION_ID; - this.consumerId = DEFAULT_CONSUMER_ID; - this.offsetData = offsetData; - } - - /** - * Constructor for version 1. - * @param groupId - * @param generationId - * @param consumerId - * @param offsetData - */ - public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map offsetData) { - super(new Struct(curSchema)); - - initCommonFields(groupId, offsetData); - struct.set(GENERATION_ID_KEY_NAME, generationId); - struct.set(CONSUMER_ID_KEY_NAME, consumerId); - this.groupId = groupId; - this.generationId = generationId; - this.consumerId = consumerId; - this.offsetData = offsetData; - } - - private void initCommonFields(String groupId, Map offsetData) { - Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); - - struct.set(GROUP_ID_KEY_NAME, groupId); - List topicArray = new ArrayList(); - for (Map.Entry> topicEntry: topicsData.entrySet()) { - Struct topicData = struct.instance(TOPICS_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); - List partitionArray = new ArrayList(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { - PartitionData fetchPartitionData = partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); - partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp); - partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(TOPICS_KEY_NAME, topicArray.toArray()); - } - - public OffsetCommitRequest(Struct struct) { - super(struct); - offsetData = new HashMap(); - for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) { - Struct topicResponse = (Struct) topicResponseObj; - String topic = topicResponse.getString(TOPIC_KEY_NAME); - for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { - Struct partitionResponse = (Struct) partitionResponseObj; - int partition = partitionResponse.getInt(PARTITION_KEY_NAME); - long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME); - long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME); - String metadata = partitionResponse.getString(METADATA_KEY_NAME); - PartitionData partitionData = new PartitionData(offset, timestamp, metadata); - offsetData.put(new TopicPartition(topic, partition), partitionData); - } - } - groupId = struct.getString(GROUP_ID_KEY_NAME); - // This field only exists in v1. - if (struct.hasField(GENERATION_ID_KEY_NAME)) - generationId = struct.getInt(GENERATION_ID_KEY_NAME); - else - generationId = DEFAULT_GENERATION_ID; - - // This field only exists in v1. - if (struct.hasField(CONSUMER_ID_KEY_NAME)) - consumerId = struct.getString(CONSUMER_ID_KEY_NAME); - else - consumerId = DEFAULT_CONSUMER_ID; - } - - public String groupId() { - return groupId; - } - - public int generationId() { - return generationId; - } - - public String consumerId() { - return consumerId; - } - - public Map offsetData() { - return offsetData; - } - - public static OffsetCommitRequest parse(ByteBuffer buffer, int versionId) { - Schema schema = ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, versionId); - return new OffsetCommitRequest(((Struct) schema.read(buffer))); - } - - public static OffsetCommitRequest parse(ByteBuffer buffer) { - return new OffsetCommitRequest(((Struct) curSchema.read(buffer))); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java deleted file mode 100644 index 711232a..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ /dev/null @@ -1,87 +0,0 @@ -/** - * 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.TopicPartition; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.utils.CollectionUtils; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class OffsetCommitResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_COMMIT.id); - private static String RESPONSES_KEY_NAME = "responses"; - - // topic level fields - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; - - // partition level fields - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; - - private final Map responseData; - - public OffsetCommitResponse(Map responseData) { - super(new Struct(curSchema)); - - Map> topicsData = CollectionUtils.groupDataByTopic(responseData); - - List topicArray = new ArrayList(); - for (Map.Entry> entries: topicsData.entrySet()) { - Struct topicData = struct.instance(RESPONSES_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, entries.getKey()); - List partitionArray = new ArrayList(); - for (Map.Entry partitionEntry : entries.getValue().entrySet()) { - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(ERROR_CODE_KEY_NAME, partitionEntry.getValue()); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); - this.responseData = responseData; - } - - public OffsetCommitResponse(Struct struct) { - super(struct); - responseData = new HashMap(); - for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { - Struct topicResponse = (Struct) topicResponseObj; - String topic = topicResponse.getString(TOPIC_KEY_NAME); - for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { - Struct partitionResponse = (Struct) partitionResponseObj; - int partition = partitionResponse.getInt(PARTITION_KEY_NAME); - short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME); - responseData.put(new TopicPartition(topic, partition), errorCode); - } - } - } - - public Map responseData() { - return responseData; - } - - public static OffsetCommitResponse parse(ByteBuffer buffer) { - return new OffsetCommitResponse(((Struct) curSchema.read(buffer))); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java deleted file mode 100644 index 90d5135..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java +++ /dev/null @@ -1,98 +0,0 @@ -/** - * 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.TopicPartition; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.utils.CollectionUtils; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -/** - * This wrapper supports both v0 and v1 of OffsetFetchRequest. - */ -public class OffsetFetchRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_FETCH.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String TOPICS_KEY_NAME = "topics"; - - // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; - - // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - - public static final int DEFAULT_GENERATION_ID = -1; - public static final String DEFAULT_CONSUMER_ID = ""; - - private final String groupId; - private final List partitions; - - public OffsetFetchRequest(String groupId, List partitions) { - super(new Struct(curSchema)); - - Map> topicsData = CollectionUtils.groupDataByTopic(partitions); - - struct.set(GROUP_ID_KEY_NAME, groupId); - List topicArray = new ArrayList(); - for (Map.Entry> entries: topicsData.entrySet()) { - Struct topicData = struct.instance(TOPICS_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, entries.getKey()); - List partitionArray = new ArrayList(); - for (Integer partiitonId : entries.getValue()) { - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partiitonId); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(TOPICS_KEY_NAME, topicArray.toArray()); - this.groupId = groupId; - this.partitions = partitions; - } - - public OffsetFetchRequest(Struct struct) { - super(struct); - partitions = new ArrayList(); - for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) { - Struct topicResponse = (Struct) topicResponseObj; - String topic = topicResponse.getString(TOPIC_KEY_NAME); - for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { - Struct partitionResponse = (Struct) partitionResponseObj; - int partition = partitionResponse.getInt(PARTITION_KEY_NAME); - partitions.add(new TopicPartition(topic, partition)); - } - } - groupId = struct.getString(GROUP_ID_KEY_NAME); - } - - public String groupId() { - return groupId; - } - - public List partitions() { - return partitions; - } - - public static OffsetFetchRequest parse(ByteBuffer buffer) { - return new OffsetFetchRequest(((Struct) curSchema.read(buffer))); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java deleted file mode 100644 index 6b7c269..0000000 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ /dev/null @@ -1,107 +0,0 @@ -/** - * 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.TopicPartition; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.utils.CollectionUtils; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class OffsetFetchResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_FETCH.id); - private static String RESPONSES_KEY_NAME = "responses"; - - // topic level fields - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; - - // partition level fields - private static String PARTITION_KEY_NAME = "partition"; - private static String COMMIT_OFFSET_KEY_NAME = "offset"; - private static String METADATA_KEY_NAME = "metadata"; - private static String ERROR_CODE_KEY_NAME = "error_code"; - - private final Map responseData; - - public static final class PartitionData { - public final long offset; - public final String metadata; - public final short errorCode; - - public PartitionData(long offset, String metadata, short errorCode) { - this.offset = offset; - this.metadata = metadata; - this.errorCode = errorCode; - } - } - - public OffsetFetchResponse(Map responseData) { - super(new Struct(curSchema)); - - Map> topicsData = CollectionUtils.groupDataByTopic(responseData); - - List topicArray = new ArrayList(); - for (Map.Entry> entries: topicsData.entrySet()) { - Struct topicData = struct.instance(RESPONSES_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, entries.getKey()); - List partitionArray = new ArrayList(); - for (Map.Entry partitionEntry : entries.getValue().entrySet()) { - PartitionData fetchPartitionData = partitionEntry.getValue(); - Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); - partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); - partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); - partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.errorCode); - partitionArray.add(partitionData); - } - topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); - } - struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); - this.responseData = responseData; - } - - public OffsetFetchResponse(Struct struct) { - super(struct); - responseData = new HashMap(); - for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { - Struct topicResponse = (Struct) topicResponseObj; - String topic = topicResponse.getString(TOPIC_KEY_NAME); - for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { - Struct partitionResponse = (Struct) partitionResponseObj; - int partition = partitionResponse.getInt(PARTITION_KEY_NAME); - long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME); - String metadata = partitionResponse.getString(METADATA_KEY_NAME); - short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME); - PartitionData partitionData = new PartitionData(offset, metadata, errorCode); - responseData.put(new TopicPartition(topic, partition), partitionData); - } - } - } - - public Map responseData() { - return responseData; - } - - public static OffsetFetchResponse parse(ByteBuffer buffer) { - return new OffsetFetchResponse(((Struct) curSchema.read(buffer))); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 3dbba8a..6036f6a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -1,105 +1,71 @@ -/** - * 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.TopicPartition; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.utils.CollectionUtils; - import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -public class ProduceRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id); - private static String ACKS_KEY_NAME = "acks"; - private static String TIMEOUT_KEY_NAME = "timeout"; - private static String TOPIC_DATA_KEY_NAME = "topic_data"; - - // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITION_DATA_KEY_NAME = "data"; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.record.MemoryRecords; - // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String RECORD_SET_KEY_NAME = "record_set"; +public class ProduceRequest { private final short acks; private final int timeout; - private final Map partitionRecords; + private final Map> records; - public ProduceRequest(short acks, int timeout, Map partitionRecords) { - super(new Struct(curSchema)); - Map> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords); - struct.set(ACKS_KEY_NAME, acks); - struct.set(TIMEOUT_KEY_NAME, timeout); - List topicDatas = new ArrayList(recordsByTopic.size()); - for (Map.Entry> entry : recordsByTopic.entrySet()) { - Struct topicData = struct.instance(TOPIC_DATA_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, entry.getKey()); - List partitionArray = new ArrayList(); - for (Map.Entry partitionEntry : entry.getValue().entrySet()) { - ByteBuffer buffer = partitionEntry.getValue().duplicate(); - Struct part = topicData.instance(PARTITION_DATA_KEY_NAME) - .set(PARTITION_KEY_NAME, partitionEntry.getKey()) - .set(RECORD_SET_KEY_NAME, buffer); - partitionArray.add(part); - } - topicData.set(PARTITION_DATA_KEY_NAME, partitionArray.toArray()); - topicDatas.add(topicData); - } - struct.set(TOPIC_DATA_KEY_NAME, topicDatas.toArray()); + public ProduceRequest(short acks, int timeout) { this.acks = acks; this.timeout = timeout; - this.partitionRecords = partitionRecords; + this.records = new HashMap>(); } - public ProduceRequest(Struct struct) { - super(struct); - partitionRecords = new HashMap(); - for (Object topicDataObj : struct.getArray(TOPIC_DATA_KEY_NAME)) { - Struct topicData = (Struct) topicDataObj; - String topic = topicData.getString(TOPIC_KEY_NAME); - for (Object partitionResponseObj : topicData.getArray(PARTITION_DATA_KEY_NAME)) { - Struct partitionResponse = (Struct) partitionResponseObj; - int partition = partitionResponse.getInt(PARTITION_KEY_NAME); - ByteBuffer records = partitionResponse.getBytes(RECORD_SET_KEY_NAME); - partitionRecords.put(new TopicPartition(topic, partition), records); - } + public void add(TopicPartition tp, MemoryRecords recs) { + List found = this.records.get(tp.topic()); + if (found == null) { + found = new ArrayList(); + records.put(tp.topic(), found); } - acks = struct.getShort(ACKS_KEY_NAME); - timeout = struct.getInt(TIMEOUT_KEY_NAME); + found.add(new PartitionRecords(tp, recs)); } - public short acks() { - return acks; + public Struct toStruct() { + Struct produce = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id)); + produce.set("acks", acks); + produce.set("timeout", timeout); + List topicDatas = new ArrayList(records.size()); + for (Map.Entry> entry : records.entrySet()) { + Struct topicData = produce.instance("topic_data"); + topicData.set("topic", entry.getKey()); + List parts = entry.getValue(); + Object[] partitionData = new Object[parts.size()]; + for (int i = 0; i < parts.size(); i++) { + ByteBuffer buffer = parts.get(i).records.buffer(); + buffer.flip(); + Struct part = topicData.instance("data") + .set("partition", parts.get(i).topicPartition.partition()) + .set("record_set", buffer); + partitionData[i] = part; + } + topicData.set("data", partitionData); + topicDatas.add(topicData); + } + produce.set("topic_data", topicDatas.toArray()); + return produce; } - public int timeout() { - return timeout; - } + private static final class PartitionRecords { + public final TopicPartition topicPartition; + public final MemoryRecords records; - public Map partitionRecords() { - return partitionRecords; + public PartitionRecords(TopicPartition topicPartition, MemoryRecords records) { + this.topicPartition = topicPartition; + this.records = records; + } } - public static ProduceRequest parse(ByteBuffer buffer) { - return new ProduceRequest(((Struct) curSchema.read(buffer))); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 5220464..6cf4fb7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -12,83 +12,67 @@ */ package org.apache.kafka.common.requests; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ProtoUtils; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.utils.CollectionUtils; - -import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; -public class ProduceResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id); - private static String RESPONSES_KEY_NAME = "responses"; - - // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITION_RESPONSES_KEY_NAME = "partition_responses"; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.types.Struct; - // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String BASE_OFFSET_KEY_NAME = "base_offset"; +public class ProduceResponse { private final Map responses; - public ProduceResponse(Map responses) { - super(new Struct(curSchema)); - Map> responseByTopic = CollectionUtils.groupDataByTopic(responses); - List topicDatas = new ArrayList(responseByTopic.size()); - for (Map.Entry> entry : responseByTopic.entrySet()) { - Struct topicData = struct.instance(RESPONSES_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, entry.getKey()); - List partitionArray = new ArrayList(); - for (Map.Entry partitionEntry : entry.getValue().entrySet()) { - PartitionResponse part = partitionEntry.getValue(); - Struct partStruct = topicData.instance(PARTITION_RESPONSES_KEY_NAME) - .set(PARTITION_KEY_NAME, partitionEntry.getKey()) - .set(ERROR_CODE_KEY_NAME, part.errorCode) - .set(BASE_OFFSET_KEY_NAME, part.baseOffset); - partitionArray.add(partStruct); - } - topicData.set(PARTITION_RESPONSES_KEY_NAME, partitionArray.toArray()); - topicDatas.add(topicData); - } - struct.set(RESPONSES_KEY_NAME, topicDatas.toArray()); - this.responses = responses; + public ProduceResponse() { + this.responses = new HashMap(); } public ProduceResponse(Struct struct) { - super(struct); responses = new HashMap(); - for (Object topicResponse : struct.getArray("responses")) { + for (Object topicResponse : (Object[]) struct.get("responses")) { Struct topicRespStruct = (Struct) topicResponse; - String topic = topicRespStruct.getString("topic"); - for (Object partResponse : topicRespStruct.getArray("partition_responses")) { + String topic = (String) topicRespStruct.get("topic"); + for (Object partResponse : (Object[]) topicRespStruct.get("partition_responses")) { Struct partRespStruct = (Struct) partResponse; - int partition = partRespStruct.getInt("partition"); - short errorCode = partRespStruct.getShort("error_code"); - long offset = partRespStruct.getLong("base_offset"); + int partition = (Integer) partRespStruct.get("partition"); + short errorCode = (Short) partRespStruct.get("error_code"); + long offset = (Long) partRespStruct.get("base_offset"); TopicPartition tp = new TopicPartition(topic, partition); - responses.put(tp, new PartitionResponse(errorCode, offset)); + responses.put(tp, new PartitionResponse(partition, errorCode, offset)); } } } + public void addResponse(TopicPartition tp, int partition, short error, long baseOffset) { + this.responses.put(tp, new PartitionResponse(partition, error, baseOffset)); + } + public Map responses() { return this.responses; } - public static final class PartitionResponse { + @Override + public String toString() { + StringBuilder b = new StringBuilder(); + b.append('{'); + boolean isFirst = true; + for (Map.Entry entry : responses.entrySet()) { + if (isFirst) + isFirst = false; + else + b.append(','); + b.append(entry.getKey() + " : " + entry.getValue()); + } + b.append('}'); + return b.toString(); + } + + public static class PartitionResponse { + public int partitionId; public short errorCode; public long baseOffset; - public PartitionResponse(short errorCode, long baseOffset) { + public PartitionResponse(int partitionId, short errorCode, long baseOffset) { + this.partitionId = partitionId; this.errorCode = errorCode; this.baseOffset = baseOffset; } @@ -97,7 +81,9 @@ public class ProduceResponse extends AbstractRequestResponse { public String toString() { StringBuilder b = new StringBuilder(); b.append('{'); - b.append("error: "); + b.append("pid: "); + b.append(partitionId); + b.append(",error: "); b.append(errorCode); b.append(",offset: "); b.append(baseOffset); @@ -105,8 +91,4 @@ public class ProduceResponse extends AbstractRequestResponse { return b.toString(); } } - - public static ProduceResponse parse(ByteBuffer buffer) { - return new ProduceResponse(((Struct) curSchema.read(buffer))); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java index f459a2a..66cc2fe 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java @@ -24,24 +24,18 @@ import org.apache.kafka.common.protocol.types.Struct; /** * The header for a request in the Kafka protocol */ -public class RequestHeader extends AbstractRequestResponse { +public class RequestHeader { private static Field API_KEY_FIELD = REQUEST_HEADER.get("api_key"); private static Field API_VERSION_FIELD = REQUEST_HEADER.get("api_version"); private static Field CLIENT_ID_FIELD = REQUEST_HEADER.get("client_id"); private static Field CORRELATION_ID_FIELD = REQUEST_HEADER.get("correlation_id"); - private final short apiKey; - private final short apiVersion; - private final String clientId; - private final int correlationId; + private final Struct header; public RequestHeader(Struct header) { - super(header); - apiKey = struct.getShort(API_KEY_FIELD); - apiVersion = struct.getShort(API_VERSION_FIELD); - clientId = struct.getString(CLIENT_ID_FIELD); - correlationId = struct.getInt(CORRELATION_ID_FIELD); + super(); + this.header = header; } public RequestHeader(short apiKey, String client, int correlation) { @@ -49,34 +43,43 @@ public class RequestHeader extends AbstractRequestResponse { } public RequestHeader(short apiKey, short version, String client, int correlation) { - super(new Struct(Protocol.REQUEST_HEADER)); - struct.set(API_KEY_FIELD, apiKey); - struct.set(API_VERSION_FIELD, version); - struct.set(CLIENT_ID_FIELD, client); - struct.set(CORRELATION_ID_FIELD, correlation); - this.apiKey = apiKey; - this.apiVersion = version; - this.clientId = client; - this.correlationId = correlation; + this(new Struct(Protocol.REQUEST_HEADER)); + this.header.set(API_KEY_FIELD, apiKey); + this.header.set(API_VERSION_FIELD, version); + this.header.set(CLIENT_ID_FIELD, client); + this.header.set(CORRELATION_ID_FIELD, correlation); } public short apiKey() { - return apiKey; + return (Short) this.header.get(API_KEY_FIELD); } public short apiVersion() { - return apiVersion; + return (Short) this.header.get(API_VERSION_FIELD); } public String clientId() { - return clientId; + return (String) this.header.get(CLIENT_ID_FIELD); } public int correlationId() { - return correlationId; + return (Integer) this.header.get(CORRELATION_ID_FIELD); } public static RequestHeader parse(ByteBuffer buffer) { return new RequestHeader((Struct) Protocol.REQUEST_HEADER.read(buffer)); } + + public void writeTo(ByteBuffer buffer) { + header.writeTo(buffer); + } + + public int sizeOf() { + return header.sizeOf(); + } + + @Override + public String toString() { + return header.toString(); + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java index dd63853..257b828 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java @@ -28,25 +28,31 @@ import org.apache.kafka.common.protocol.types.Struct; /** * A response header in the kafka protocol. */ -public class ResponseHeader extends AbstractRequestResponse { +public class ResponseHeader { private static Field CORRELATION_KEY_FIELD = RESPONSE_HEADER.get("correlation_id"); - private final int correlationId; + private final Struct header; public ResponseHeader(Struct header) { - super(header); - correlationId = struct.getInt(CORRELATION_KEY_FIELD); + this.header = header; } public ResponseHeader(int correlationId) { - super(new Struct(Protocol.RESPONSE_HEADER)); - struct.set(CORRELATION_KEY_FIELD, correlationId); - this.correlationId = correlationId; + this(new Struct(Protocol.RESPONSE_HEADER)); + this.header.set(CORRELATION_KEY_FIELD, correlationId); } public int correlationId() { - return correlationId; + return (Integer) header.get(CORRELATION_KEY_FIELD); + } + + public void writeTo(ByteBuffer buffer) { + header.writeTo(buffer); + } + + public int sizeOf() { + return header.sizeOf(); } public static ResponseHeader parse(ByteBuffer buffer) { diff --git a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java deleted file mode 100644 index ba38637..0000000 --- a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java +++ /dev/null @@ -1,62 +0,0 @@ -/** - * 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.utils; - -import org.apache.kafka.common.TopicPartition; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class CollectionUtils { - /** - * group data by topic - * @param data Data to be partitioned - * @param Partition data type - * @return partitioned data - */ - public static Map> groupDataByTopic(Map data) { - Map> dataByTopic = new HashMap>(); - for (Map.Entry entry: data.entrySet()) { - String topic = entry.getKey().topic(); - int partition = entry.getKey().partition(); - Map topicData = dataByTopic.get(topic); - if (topicData == null) { - topicData = new HashMap(); - dataByTopic.put(topic, topicData); - } - topicData.put(partition, entry.getValue()); - } - return dataByTopic; - } - - /** - * group partitions by topic - * @param partitions - * @return partitions per topic - */ - public static Map> groupDataByTopic(List partitions) { - Map> partitionsByTopic = new HashMap>(); - for (TopicPartition tp: partitions) { - String topic = tp.topic(); - List topicData = partitionsByTopic.get(topic); - if (topicData == null) { - topicData = new ArrayList(); - partitionsByTopic.put(topic, topicData); - } - topicData.add(tp.partition()); - } - return partitionsByTopic; - } -} diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index 1a55242..2f98192 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -7,13 +7,11 @@ import static org.junit.Assert.assertTrue; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; import org.apache.kafka.clients.producer.internals.Metadata; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ProtoUtils; @@ -70,7 +68,7 @@ public class NetworkClientTest { @Test public void testSimpleRequestResponse() { - ProduceRequest produceRequest = new ProduceRequest((short) 1, 1000, Collections.emptyMap()); + ProduceRequest produceRequest = new ProduceRequest((short) 1, 1000); RequestHeader reqHeader = client.nextRequestHeader(ApiKeys.PRODUCE); RequestSend send = new RequestSend(node.id(), reqHeader, produceRequest.toStruct()); ClientRequest request = new ClientRequest(time.milliseconds(), true, send, null); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java deleted file mode 100644 index df37fc6..0000000 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ /dev/null @@ -1,173 +0,0 @@ -/** - * 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.Cluster; -import org.apache.kafka.common.Node; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; -import org.junit.Test; - -import java.lang.reflect.Method; -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.junit.Assert.assertEquals; - -public class RequestResponseTest { - - @Test - public void testSerialization() throws Exception{ - List requestList = Arrays.asList( - createRequestHeader(), - createResponseHeader(), - createConsumerMetadataRequest(), - createConsumerMetadataResponse(), - createFetchRequest(), - createFetchResponse(), - createHeartBeatRequest(), - createHeartBeatResponse(), - createJoinGroupRequest(), - createJoinGroupResponse(), - createListOffsetRequest(), - createListOffsetResponse(), - createMetadataRequest(), - createMetadataResponse(), - createOffsetCommitRequest(), - createOffsetCommitResponse(), - createOffsetFetchRequest(), - createOffsetFetchResponse(), - createProduceRequest(), - createProduceResponse()); - - for (AbstractRequestResponse req: requestList) { - ByteBuffer buffer = ByteBuffer.allocate(req.sizeOf()); - req.writeTo(buffer); - buffer.rewind(); - Method deserializer = req.getClass().getDeclaredMethod("parse", ByteBuffer.class); - AbstractRequestResponse deserialized = (AbstractRequestResponse) deserializer.invoke(null, buffer); - assertEquals("The original and deserialized of " + req.getClass().getSimpleName() + " should be the same.", req, deserialized); - assertEquals("The original and deserialized of " + req.getClass().getSimpleName() + " should have the same hashcode.", - req.hashCode(), deserialized.hashCode()); - } - } - - private AbstractRequestResponse createRequestHeader() { - return new RequestHeader((short)10, (short)1, "", 10); - } - - private AbstractRequestResponse createResponseHeader() { - return new ResponseHeader(10); - } - - private AbstractRequestResponse createConsumerMetadataRequest() { - return new ConsumerMetadataRequest("test-group"); - } - - private AbstractRequestResponse createConsumerMetadataResponse() { - return new ConsumerMetadataResponse((short)1, new Node(10, "host1", 2014)); - } - - private AbstractRequestResponse createFetchRequest() { - Map fetchData = new HashMap(); - fetchData.put(new TopicPartition("test1", 0), new FetchRequest.PartitionData(100, 1000000)); - fetchData.put(new TopicPartition("test2", 0), new FetchRequest.PartitionData(200, 1000000)); - return new FetchRequest(-1, 100, 100000, fetchData); - } - - private AbstractRequestResponse createFetchResponse() { - Map responseData = new HashMap(); - responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData((short)0, 1000000, ByteBuffer.allocate(10))); - return new FetchResponse(responseData); - } - - private AbstractRequestResponse createHeartBeatRequest() { - return new HeartbeatRequest("group1", 1, "consumer1"); - } - - private AbstractRequestResponse createHeartBeatResponse() { - return new HeartbeatResponse((short)0); - } - - private AbstractRequestResponse createJoinGroupRequest() { - return new JoinGroupRequest("group1", 30000, Arrays.asList("topic1"), "consumer1", "strategy1"); - } - - private AbstractRequestResponse createJoinGroupResponse() { - return new JoinGroupResponse((short)0, 1, "consumer1", Arrays.asList(new TopicPartition("test11", 1), new TopicPartition("test2", 1))); - } - - private AbstractRequestResponse createListOffsetRequest() { - Map offsetData = new HashMap(); - offsetData.put(new TopicPartition("test", 0), new ListOffsetRequest.PartitionData(1000000L, 10)); - return new ListOffsetRequest(-1, offsetData); - } - - private AbstractRequestResponse createListOffsetResponse() { - Map responseData = new HashMap(); - responseData.put(new TopicPartition("test", 0), new ListOffsetResponse.PartitionData((short)0, Arrays.asList(100L))); - return new ListOffsetResponse(responseData); - } - - private AbstractRequestResponse createMetadataRequest() { - return new MetadataRequest(Arrays.asList("topic1")); - } - - private AbstractRequestResponse createMetadataResponse() { - Node node = new Node(1, "host1", 1001); - Node[] replicas = new Node[1]; - replicas[0] = node; - Node[] isr = new Node[1]; - isr[0] = node; - Cluster cluster = new Cluster(Arrays.asList(node), Arrays.asList(new PartitionInfo("topic1", 1, node, replicas, isr))); - return new MetadataResponse(cluster); - } - - private AbstractRequestResponse createOffsetCommitRequest() { - Map commitData = new HashMap(); - commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionData(100, 1000000, "")); - return new OffsetCommitRequest("group1", 100, "consumer1", commitData); - } - - private AbstractRequestResponse createOffsetCommitResponse() { - Map responseData = new HashMap(); - responseData.put(new TopicPartition("test", 0), (short)0); - return new OffsetCommitResponse(responseData); - } - - private AbstractRequestResponse createOffsetFetchRequest() { - return new OffsetFetchRequest("group1", Arrays.asList(new TopicPartition("test11", 1))); - } - - private AbstractRequestResponse createOffsetFetchResponse() { - Map responseData = new HashMap(); - responseData.put(new TopicPartition("test", 0), new OffsetFetchResponse.PartitionData(100L, "", (short)0)); - return new OffsetFetchResponse(responseData); - } - - private AbstractRequestResponse createProduceRequest() { - Map produceData = new HashMap(); - produceData.put(new TopicPartition("test", 0), ByteBuffer.allocate(10)); - return new ProduceRequest((short)0, 5000, produceData); - } - - private AbstractRequestResponse createProduceResponse() { - Map responseData = new HashMap(); - responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse((short) 0, 10000)); - return new ProduceResponse(responseData); - } -} diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala index 6d00ed0..dfad6e6 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala @@ -41,9 +41,9 @@ object ConsumerMetadataRequest { case class ConsumerMetadataRequest(group: String, versionId: Short = ConsumerMetadataRequest.CurrentVersion, - correlationId: Int = 0, + override val correlationId: Int = 0, clientId: String = ConsumerMetadataRequest.DefaultClientId) - extends RequestOrResponse(Some(RequestKeys.ConsumerMetadataKey)) { + extends RequestOrResponse(Some(RequestKeys.ConsumerMetadataKey), correlationId) { def sizeInBytes = 2 + /* versionId */ diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala index 84f6017..c72ca14 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala @@ -40,8 +40,8 @@ object ConsumerMetadataResponse { } -case class ConsumerMetadataResponse (coordinatorOpt: Option[Broker], errorCode: Short, correlationId: Int = 0) - extends RequestOrResponse() { +case class ConsumerMetadataResponse (coordinatorOpt: Option[Broker], errorCode: Short, override val correlationId: Int = 0) + extends RequestOrResponse(correlationId = correlationId) { def sizeInBytes = 4 + /* correlationId */ diff --git a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala index 5be393a..7dacb20 100644 --- a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala +++ b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala @@ -38,9 +38,9 @@ object ControlledShutdownRequest extends Logging { } case class ControlledShutdownRequest(val versionId: Short, - val correlationId: Int, + override val correlationId: Int, val brokerId: Int) - extends RequestOrResponse(Some(RequestKeys.ControlledShutdownKey)){ + extends RequestOrResponse(Some(RequestKeys.ControlledShutdownKey), correlationId){ def this(correlationId: Int, brokerId: Int) = this(ControlledShutdownRequest.CurrentVersion, correlationId, brokerId) diff --git a/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala b/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala index 5e0a1cf..46ec3db 100644 --- a/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala +++ b/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala @@ -39,10 +39,10 @@ object ControlledShutdownResponse { } -case class ControlledShutdownResponse(val correlationId: Int, +case class ControlledShutdownResponse(override val correlationId: Int, val errorCode: Short = ErrorMapping.NoError, val partitionsRemaining: Set[TopicAndPartition]) - extends RequestOrResponse() { + extends RequestOrResponse(correlationId = correlationId) { def sizeInBytes(): Int ={ var size = 4 /* correlation id */ + diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index 55a5982..a8b73ac 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -60,13 +60,13 @@ object FetchRequest { } case class FetchRequest private[kafka] (versionId: Short = FetchRequest.CurrentVersion, - correlationId: Int = FetchRequest.DefaultCorrelationId, + override val correlationId: Int = FetchRequest.DefaultCorrelationId, clientId: String = ConsumerConfig.DefaultClientId, replicaId: Int = Request.OrdinaryConsumerId, maxWait: Int = FetchRequest.DefaultMaxWait, minBytes: Int = FetchRequest.DefaultMinBytes, requestInfo: Map[TopicAndPartition, PartitionFetchInfo]) - extends RequestOrResponse(Some(RequestKeys.FetchKey)) { + extends RequestOrResponse(Some(RequestKeys.FetchKey), correlationId) { /** * Partitions the request info into a map of maps (one for each topic). diff --git a/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala b/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala deleted file mode 100644 index fb022e8..0000000 --- a/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala +++ /dev/null @@ -1,45 +0,0 @@ -/** - * 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 kafka.api - -import java.nio.ByteBuffer -import org.apache.kafka.common.requests.AbstractRequestResponse - -private[kafka] abstract class GenericRequestOrResponseAndHeader(val header: AbstractRequestResponse, - val body: AbstractRequestResponse, - val name: String, - override val requestId: Option[Short] = None) - extends RequestOrResponse(requestId) { - - def writeTo(buffer: ByteBuffer) { - header.writeTo(buffer) - body.writeTo(buffer) - } - - def sizeInBytes(): Int = { - header.sizeOf() + body.sizeOf(); - } - - override def toString(): String = { - describe(true) - } - - override def describe(details: Boolean): String = { - val strBuffer = new StringBuilder - strBuffer.append("Name: " + name) - strBuffer.append("; header: " + header.toString) - strBuffer.append("; body: " + body.toString) - strBuffer.toString() - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala b/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala deleted file mode 100644 index 932418b..0000000 --- a/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala +++ /dev/null @@ -1,39 +0,0 @@ -/** - * 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 kafka.api - -import java.nio.ByteBuffer -import kafka.network.{BoundedByteBufferSend, RequestChannel} -import kafka.common.ErrorMapping -import kafka.network.RequestChannel.Response -import org.apache.kafka.common.requests.{HeartbeatResponse, ResponseHeader, HeartbeatRequest, RequestHeader} - -object HeartbeatRequestAndHeader { - def readFrom(buffer: ByteBuffer): HeartbeatRequestAndHeader = { - val header = RequestHeader.parse(buffer) - val body = HeartbeatRequest.parse(buffer) - new HeartbeatRequestAndHeader(header, body) - } -} - -case class HeartbeatRequestAndHeader(override val header: RequestHeader, override val body: HeartbeatRequest) - extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), Some(RequestKeys.HeartbeatKey)) { - - override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { - val errorResponseHeader = new ResponseHeader(header.correlationId) - val errorResponseBody = new HeartbeatResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) - val errorHeartBeatResponseAndHeader = new HeartbeatResponseAndHeader(errorResponseHeader, errorResponseBody) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorHeartBeatResponseAndHeader))) - } -} diff --git a/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala b/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala deleted file mode 100644 index 556f38d..0000000 --- a/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala +++ /dev/null @@ -1,28 +0,0 @@ -/** - * 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 kafka.api - -import org.apache.kafka.common.requests.{ResponseHeader, HeartbeatResponse} -import java.nio.ByteBuffer - -object HeartbeatResponseAndHeader { - def readFrom(buffer: ByteBuffer): HeartbeatResponseAndHeader = { - val header = ResponseHeader.parse(buffer) - val body = HeartbeatResponse.parse(buffer) - new HeartbeatResponseAndHeader(header, body) - } -} - -case class HeartbeatResponseAndHeader(override val header: ResponseHeader, override val body: HeartbeatResponse) - extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), None) { -} diff --git a/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala b/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala deleted file mode 100644 index 9aea28c..0000000 --- a/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala +++ /dev/null @@ -1,40 +0,0 @@ -/** - * 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 kafka.api - -import java.nio.ByteBuffer -import kafka.network.{BoundedByteBufferSend, RequestChannel} -import kafka.common.ErrorMapping -import org.apache.kafka.common.requests._ -import kafka.network.RequestChannel.Response -import scala.Some - -object JoinGroupRequestAndHeader { - def readFrom(buffer: ByteBuffer): JoinGroupRequestAndHeader = { - val header = RequestHeader.parse(buffer) - val body = JoinGroupRequest.parse(buffer) - new JoinGroupRequestAndHeader(header, body) - } -} - -case class JoinGroupRequestAndHeader(override val header: RequestHeader, override val body: JoinGroupRequest) - extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), Some(RequestKeys.JoinGroupKey)) { - - override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { - val errorResponseHeader = new ResponseHeader(header.correlationId) - val errorResponseBody = new JoinGroupResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) - val errorHeartBeatResponseAndHeader = new JoinGroupResponseAndHeader(errorResponseHeader, errorResponseBody) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorHeartBeatResponseAndHeader))) - } -} diff --git a/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala b/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala deleted file mode 100644 index 7389ae6..0000000 --- a/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala +++ /dev/null @@ -1,28 +0,0 @@ -/** - * 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 kafka.api - -import org.apache.kafka.common.requests.{JoinGroupResponse, ResponseHeader} -import java.nio.ByteBuffer - -object JoinGroupResponseAndHeader { - def readFrom(buffer: ByteBuffer): JoinGroupResponseAndHeader = { - val header = ResponseHeader.parse(buffer) - val body = JoinGroupResponse.parse(buffer) - new JoinGroupResponseAndHeader(header, body) - } -} - -case class JoinGroupResponseAndHeader(override val header: ResponseHeader, override val body: JoinGroupResponse) - extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), None) { -} diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala index 4ff7e8f..3e40817 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala @@ -129,13 +129,13 @@ object LeaderAndIsrRequest { } case class LeaderAndIsrRequest (versionId: Short, - correlationId: Int, + override val correlationId: Int, clientId: String, controllerId: Int, controllerEpoch: Int, partitionStateInfos: Map[(String, Int), PartitionStateInfo], leaders: Set[Broker]) - extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey)) { + extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey), correlationId) { def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], leaders: Set[Broker], controllerId: Int, controllerEpoch: Int, correlationId: Int, clientId: String) = { diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala b/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala index 22ce48a..f636444 100644 --- a/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala +++ b/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala @@ -41,10 +41,10 @@ object LeaderAndIsrResponse { } -case class LeaderAndIsrResponse(correlationId: Int, +case class LeaderAndIsrResponse(override val correlationId: Int, responseMap: Map[(String, Int), Short], errorCode: Short = ErrorMapping.NoError) - extends RequestOrResponse() { + extends RequestOrResponse(correlationId = correlationId) { def sizeInBytes(): Int ={ var size = 4 /* correlation id */ + diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index 861a6cf..630768a 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -26,7 +26,7 @@ import kafka.network.RequestChannel.Response import scala.collection._ object OffsetCommitRequest extends Logging { - val CurrentVersion: Short = 1 + val CurrentVersion: Short = 0 val DefaultClientId = "" def readFrom(buffer: ByteBuffer): OffsetCommitRequest = { @@ -34,23 +34,11 @@ object OffsetCommitRequest extends Logging { // Read values from the envelope val versionId = buffer.getShort - assert(versionId == 0 || versionId == 1, - "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0 or 1.") - val correlationId = buffer.getInt val clientId = readShortString(buffer) // Read the OffsetRequest val consumerGroupId = readShortString(buffer) - - // version 1 specific fields - var groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID - var consumerId: String = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID - if (versionId == 1) { - groupGenerationId = buffer.getInt - consumerId = readShortString(buffer) - } - val topicCount = buffer.getInt val pairs = (1 to topicCount).flatMap(_ => { val topic = readShortString(buffer) @@ -66,20 +54,16 @@ object OffsetCommitRequest extends Logging { (TopicAndPartition(topic, partitionId), OffsetAndMetadata(offset, metadata, timestamp)) }) }) - OffsetCommitRequest(consumerGroupId, immutable.Map(pairs:_*), versionId, correlationId, clientId, groupGenerationId, consumerId) + OffsetCommitRequest(consumerGroupId, immutable.Map(pairs:_*), versionId, correlationId, clientId) } } case class OffsetCommitRequest(groupId: String, requestInfo: immutable.Map[TopicAndPartition, OffsetAndMetadata], versionId: Short = OffsetCommitRequest.CurrentVersion, - correlationId: Int = 0, - clientId: String = OffsetCommitRequest.DefaultClientId, - groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID, - consumerId: String = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID) - extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey)) { - assert(versionId == 0 || versionId == 1, - "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0 or 1.") + override val correlationId: Int = 0, + clientId: String = OffsetCommitRequest.DefaultClientId) + extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey), correlationId) { lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) @@ -100,6 +84,7 @@ case class OffsetCommitRequest(groupId: String, OffsetCommitResponse(commitStatus, correlationId) } + def writeTo(buffer: ByteBuffer) { // Write envelope buffer.putShort(versionId) @@ -108,12 +93,6 @@ case class OffsetCommitRequest(groupId: String, // Write OffsetCommitRequest writeShortString(buffer, groupId) // consumer group - - // version 1 specific data - if (versionId == 1) { - buffer.putInt(groupGenerationId) - writeShortString(buffer, consumerId) - } buffer.putInt(requestInfoGroupedByTopic.size) // number of topics requestInfoGroupedByTopic.foreach( t1 => { // topic -> Map[TopicAndPartition, OffsetMetadataAndError] writeShortString(buffer, t1._1) // topic @@ -131,8 +110,7 @@ case class OffsetCommitRequest(groupId: String, 2 + /* versionId */ 4 + /* correlationId */ shortStringLength(clientId) + - shortStringLength(groupId) + - (if (versionId == 1) 4 /* group generation id */ + shortStringLength(consumerId) else 0) + + shortStringLength(groupId) + 4 + /* topic count */ requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => { val (topic, offsets) = topicAndOffsets @@ -161,8 +139,6 @@ case class OffsetCommitRequest(groupId: String, offsetCommitRequest.append("; CorrelationId: " + correlationId) offsetCommitRequest.append("; ClientId: " + clientId) offsetCommitRequest.append("; GroupId: " + groupId) - offsetCommitRequest.append("; GroupGenerationId: " + groupGenerationId) - offsetCommitRequest.append("; ConsumerId: " + consumerId) if(details) offsetCommitRequest.append("; RequestInfo: " + requestInfo.mkString(",")) offsetCommitRequest.toString() diff --git a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala index 624a1c1..4946e97 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala @@ -42,8 +42,8 @@ object OffsetCommitResponse extends Logging { } case class OffsetCommitResponse(commitStatus: Map[TopicAndPartition, Short], - correlationId: Int = 0) - extends RequestOrResponse() { + override val correlationId: Int = 0) + extends RequestOrResponse(correlationId=correlationId) { lazy val commitStatusGroupedByTopic = commitStatus.groupBy(_._1.topic) diff --git a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala index c7604b9..a32f858 100644 --- a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala @@ -52,9 +52,9 @@ object OffsetFetchRequest extends Logging { case class OffsetFetchRequest(groupId: String, requestInfo: Seq[TopicAndPartition], versionId: Short = OffsetFetchRequest.CurrentVersion, - correlationId: Int = 0, + override val correlationId: Int = 0, clientId: String = OffsetFetchRequest.DefaultClientId) - extends RequestOrResponse(Some(RequestKeys.OffsetFetchKey)) { + extends RequestOrResponse(Some(RequestKeys.OffsetFetchKey), correlationId) { lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_.topic) diff --git a/core/src/main/scala/kafka/api/OffsetFetchResponse.scala b/core/src/main/scala/kafka/api/OffsetFetchResponse.scala index e3523f8..c1222f4 100644 --- a/core/src/main/scala/kafka/api/OffsetFetchResponse.scala +++ b/core/src/main/scala/kafka/api/OffsetFetchResponse.scala @@ -45,8 +45,8 @@ object OffsetFetchResponse extends Logging { } case class OffsetFetchResponse(requestInfo: Map[TopicAndPartition, OffsetMetadataAndError], - correlationId: Int = 0) - extends RequestOrResponse() { + override val correlationId: Int = 0) + extends RequestOrResponse(correlationId = correlationId) { lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) diff --git a/core/src/main/scala/kafka/api/OffsetRequest.scala b/core/src/main/scala/kafka/api/OffsetRequest.scala index 3d483bc..7cbc26c 100644 --- a/core/src/main/scala/kafka/api/OffsetRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetRequest.scala @@ -57,10 +57,10 @@ case class PartitionOffsetRequestInfo(time: Long, maxNumOffsets: Int) case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo], versionId: Short = OffsetRequest.CurrentVersion, - correlationId: Int = 0, + override val correlationId: Int = 0, clientId: String = OffsetRequest.DefaultClientId, replicaId: Int = Request.OrdinaryConsumerId) - extends RequestOrResponse(Some(RequestKeys.OffsetsKey)) { + extends RequestOrResponse(Some(RequestKeys.OffsetsKey), correlationId) { def this(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo], correlationId: Int, replicaId: Int) = this(requestInfo, OffsetRequest.CurrentVersion, correlationId, OffsetRequest.DefaultClientId, replicaId) diff --git a/core/src/main/scala/kafka/api/OffsetResponse.scala b/core/src/main/scala/kafka/api/OffsetResponse.scala index 63c0899..0e1d6e3 100644 --- a/core/src/main/scala/kafka/api/OffsetResponse.scala +++ b/core/src/main/scala/kafka/api/OffsetResponse.scala @@ -51,9 +51,9 @@ case class PartitionOffsetsResponse(error: Short, offsets: Seq[Long]) { } -case class OffsetResponse(correlationId: Int, +case class OffsetResponse(override val correlationId: Int, partitionErrorAndOffsets: Map[TopicAndPartition, PartitionOffsetsResponse]) - extends RequestOrResponse() { + extends RequestOrResponse(correlationId = correlationId) { lazy val offsetsGroupedByTopic = partitionErrorAndOffsets.groupBy(_._1.topic) diff --git a/core/src/main/scala/kafka/api/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala index b2366e7..0c295a2 100644 --- a/core/src/main/scala/kafka/api/ProducerRequest.scala +++ b/core/src/main/scala/kafka/api/ProducerRequest.scala @@ -53,12 +53,12 @@ object ProducerRequest { } case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion, - correlationId: Int, + override val correlationId: Int, clientId: String, requiredAcks: Short, ackTimeoutMs: Int, data: collection.mutable.Map[TopicAndPartition, ByteBufferMessageSet]) - extends RequestOrResponse(Some(RequestKeys.ProduceKey)) { + extends RequestOrResponse(Some(RequestKeys.ProduceKey), correlationId) { /** * Partitions the data into a map of maps (one for each topic). diff --git a/core/src/main/scala/kafka/api/ProducerResponse.scala b/core/src/main/scala/kafka/api/ProducerResponse.scala index a286272..5a1d801 100644 --- a/core/src/main/scala/kafka/api/ProducerResponse.scala +++ b/core/src/main/scala/kafka/api/ProducerResponse.scala @@ -43,9 +43,9 @@ object ProducerResponse { case class ProducerResponseStatus(var error: Short, offset: Long) -case class ProducerResponse(correlationId: Int, +case class ProducerResponse(override val correlationId: Int, status: Map[TopicAndPartition, ProducerResponseStatus]) - extends RequestOrResponse() { + extends RequestOrResponse(correlationId = correlationId) { /** * Partitions the status map into a map of maps (one for each topic). diff --git a/core/src/main/scala/kafka/api/RequestKeys.scala b/core/src/main/scala/kafka/api/RequestKeys.scala index c24c034..fbfc9d3 100644 --- a/core/src/main/scala/kafka/api/RequestKeys.scala +++ b/core/src/main/scala/kafka/api/RequestKeys.scala @@ -32,8 +32,6 @@ object RequestKeys { val OffsetCommitKey: Short = 8 val OffsetFetchKey: Short = 9 val ConsumerMetadataKey: Short = 10 - val JoinGroupKey: Short = 11 - val HeartbeatKey: Short = 12 val keyToNameAndDeserializerMap: Map[Short, (String, (ByteBuffer) => RequestOrResponse)]= Map(ProduceKey -> ("Produce", ProducerRequest.readFrom), @@ -46,10 +44,7 @@ object RequestKeys { ControlledShutdownKey -> ("ControlledShutdown", ControlledShutdownRequest.readFrom), OffsetCommitKey -> ("OffsetCommit", OffsetCommitRequest.readFrom), OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom), - ConsumerMetadataKey -> ("ConsumerMetadata", ConsumerMetadataRequest.readFrom), - JoinGroupKey -> ("JoinGroup", JoinGroupRequestAndHeader.readFrom), - HeartbeatKey -> ("Heartbeat", HeartbeatRequestAndHeader.readFrom) - ) + ConsumerMetadataKey -> ("ConsumerMetadata", ConsumerMetadataRequest.readFrom)) def nameForKey(key: Short): String = { keyToNameAndDeserializerMap.get(key) match { diff --git a/core/src/main/scala/kafka/api/RequestOrResponse.scala b/core/src/main/scala/kafka/api/RequestOrResponse.scala index 0334343..57f87a4 100644 --- a/core/src/main/scala/kafka/api/RequestOrResponse.scala +++ b/core/src/main/scala/kafka/api/RequestOrResponse.scala @@ -30,7 +30,7 @@ object Request { } -private[kafka] abstract class RequestOrResponse(val requestId: Option[Short] = None) extends Logging { +private[kafka] abstract class RequestOrResponse(val requestId: Option[Short] = None, val correlationId: Int) extends Logging { def sizeInBytes: Int diff --git a/core/src/main/scala/kafka/api/StopReplicaRequest.scala b/core/src/main/scala/kafka/api/StopReplicaRequest.scala index 5e14987..68fc138 100644 --- a/core/src/main/scala/kafka/api/StopReplicaRequest.scala +++ b/core/src/main/scala/kafka/api/StopReplicaRequest.scala @@ -54,13 +54,13 @@ object StopReplicaRequest extends Logging { } case class StopReplicaRequest(versionId: Short, - correlationId: Int, + override val correlationId: Int, clientId: String, controllerId: Int, controllerEpoch: Int, deletePartitions: Boolean, partitions: Set[TopicAndPartition]) - extends RequestOrResponse(Some(RequestKeys.StopReplicaKey)) { + extends RequestOrResponse(Some(RequestKeys.StopReplicaKey), correlationId) { def this(deletePartitions: Boolean, partitions: Set[TopicAndPartition], controllerId: Int, controllerEpoch: Int, correlationId: Int) = { this(StopReplicaRequest.CurrentVersion, correlationId, StopReplicaRequest.DefaultClientId, diff --git a/core/src/main/scala/kafka/api/StopReplicaResponse.scala b/core/src/main/scala/kafka/api/StopReplicaResponse.scala index 3431f3f..c90ddee 100644 --- a/core/src/main/scala/kafka/api/StopReplicaResponse.scala +++ b/core/src/main/scala/kafka/api/StopReplicaResponse.scala @@ -42,10 +42,10 @@ object StopReplicaResponse { } -case class StopReplicaResponse(val correlationId: Int, +case class StopReplicaResponse(override val correlationId: Int, val responseMap: Map[TopicAndPartition, Short], val errorCode: Short = ErrorMapping.NoError) - extends RequestOrResponse() { + extends RequestOrResponse(correlationId = correlationId) { def sizeInBytes(): Int ={ var size = 4 /* correlation id */ + diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala index 7dca09c..a319f2f 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala @@ -47,10 +47,10 @@ object TopicMetadataRequest extends Logging { } case class TopicMetadataRequest(val versionId: Short, - val correlationId: Int, + override val correlationId: Int, val clientId: String, val topics: Seq[String]) - extends RequestOrResponse(Some(RequestKeys.MetadataKey)){ + extends RequestOrResponse(Some(RequestKeys.MetadataKey), correlationId){ def this(topics: Seq[String], correlationId: Int) = this(TopicMetadataRequest.CurrentVersion, correlationId, TopicMetadataRequest.DefaultClientId, topics) @@ -79,7 +79,7 @@ case class TopicMetadataRequest(val versionId: Short, val topicMetadata = topics.map { topic => TopicMetadata(topic, Nil, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) } - val errorResponse = TopicMetadataResponse(Seq(), topicMetadata, correlationId) + val errorResponse = TopicMetadataResponse(topicMetadata, correlationId) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) } diff --git a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala index 92ac4e6..f6b7429 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala @@ -29,27 +29,34 @@ object TopicMetadataResponse { val brokerMap = brokers.map(b => (b.id, b)).toMap val topicCount = buffer.getInt val topicsMetadata = (0 until topicCount).map(_ => TopicMetadata.readFrom(buffer, brokerMap)) - new TopicMetadataResponse(brokers, topicsMetadata, correlationId) + new TopicMetadataResponse(topicsMetadata, correlationId) } } -case class TopicMetadataResponse(brokers: Seq[Broker], - topicsMetadata: Seq[TopicMetadata], - correlationId: Int) - extends RequestOrResponse() { +case class TopicMetadataResponse(topicsMetadata: Seq[TopicMetadata], + override val correlationId: Int) + extends RequestOrResponse(correlationId = correlationId) { val sizeInBytes: Int = { + val brokers = extractBrokers(topicsMetadata).values 4 + 4 + brokers.map(_.sizeInBytes).sum + 4 + topicsMetadata.map(_.sizeInBytes).sum } def writeTo(buffer: ByteBuffer) { buffer.putInt(correlationId) /* brokers */ + val brokers = extractBrokers(topicsMetadata).values buffer.putInt(brokers.size) brokers.foreach(_.writeTo(buffer)) /* topic metadata */ buffer.putInt(topicsMetadata.length) topicsMetadata.foreach(_.writeTo(buffer)) } + + def extractBrokers(topicMetadatas: Seq[TopicMetadata]): Map[Int, Broker] = { + val parts = topicsMetadata.flatMap(_.partitionsMetadata) + val brokers = (parts.flatMap(_.replicas)) ++ (parts.map(_.leader).collect{case Some(l) => l}) + brokers.map(b => (b.id, b)).toMap + } override def describe(details: Boolean):String = { toString } } diff --git a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala index 530982e..543e262 100644 --- a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala @@ -55,13 +55,13 @@ object UpdateMetadataRequest { } case class UpdateMetadataRequest (versionId: Short, - correlationId: Int, + override val correlationId: Int, clientId: String, controllerId: Int, controllerEpoch: Int, partitionStateInfos: Map[TopicAndPartition, PartitionStateInfo], aliveBrokers: Set[Broker]) - extends RequestOrResponse(Some(RequestKeys.UpdateMetadataKey)) { + extends RequestOrResponse(Some(RequestKeys.UpdateMetadataKey), correlationId) { def this(controllerId: Int, controllerEpoch: Int, correlationId: Int, clientId: String, partitionStateInfos: Map[TopicAndPartition, PartitionStateInfo], aliveBrokers: Set[Broker]) = { diff --git a/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala b/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala index 53f6067..c583c1f 100644 --- a/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala @@ -32,9 +32,9 @@ object UpdateMetadataResponse { } } -case class UpdateMetadataResponse(correlationId: Int, +case class UpdateMetadataResponse(override val correlationId: Int, errorCode: Short = ErrorMapping.NoError) - extends RequestOrResponse() { + extends RequestOrResponse(correlationId = correlationId) { def sizeInBytes(): Int = 4 /* correlation id */ + 2 /* error code */ def writeTo(buffer: ByteBuffer) { diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index 1cf2f62..3a2f90a 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -49,6 +49,7 @@ object ConsumerConfig extends Config { val MirrorTopicsWhitelistProp = "mirror.topics.whitelist" val MirrorTopicsBlacklistProp = "mirror.topics.blacklist" val ExcludeInternalTopics = true + val PartitionDistributionStrategy = "range" /* select between "range","roundrobin" and "symmetric" */ val MirrorConsumerNumThreadsProp = "mirror.consumer.numthreads" val DefaultClientId = "" @@ -175,6 +176,9 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( /** Whether messages from internal topics (such as offsets) should be exposed to the consumer. */ val excludeInternalTopics = props.getBoolean("exclude.internal.topics", ExcludeInternalTopics) + /** Select a strategy for assigning partitions to consumer streams. Possible values: range, roundrobin */ + val partitionDistributionStrategy = props.getString("partition.distribution.strategy", PartitionDistributionStrategy) + validate(this) } diff --git a/core/src/main/scala/kafka/consumer/PartitionAllocator.scala b/core/src/main/scala/kafka/consumer/PartitionAllocator.scala new file mode 100644 index 0000000..c446e51 --- /dev/null +++ b/core/src/main/scala/kafka/consumer/PartitionAllocator.scala @@ -0,0 +1,205 @@ +package kafka.consumer + +import org.I0Itec.zkclient.ZkClient +import kafka.common.TopicAndPartition +import kafka.utils.{Utils, ZkUtils, Logging} +import scala.collection.mutable + +trait PartitionAllocator { + + def allocate(): scala.collection.Map[TopicAndPartition, String] + +} + +class AllocatorContext(group: String, consumerId: String, excludeInternalTopics: Boolean, zkClient: ZkClient) { + val myTopicCount = TopicCount.constructTopicCount(group, consumerId, zkClient, excludeInternalTopics) + val myTopicThreadIdsMap = myTopicCount.getConsumerThreadIdsPerTopic + val partitionsPerTopicMap = { + val partitionsAssignmentPerTopicMap = ZkUtils.getPartitionAssignmentForTopics(zkClient, myTopicThreadIdsMap.keySet.toSeq) + partitionsAssignmentPerTopicMap.map(p => (p._1, p._2.keySet.toSeq.sorted)) + } +} + +/** + * Global round-robin allocation strategy - this should only be used when all consumers in the group have identical + * wildcard (i.e., not static) subscriptions. It lays out all the available partitions using a two-level ordering: + * first by partition number and then by topic. It also lays out the available wildcard consumer threads and proceeds + * to do a round-robin assignment from partition to consumer thread. The end result is that the partitions will be + * uniformly distributed. (i.e., the partition ownership counts will within a delta of exactly one across all consumer + * threads.) + * + * @param group Consumer group + * @param consumerId Consumer id of the consumer in the group for which we are doing the allocation + * @param excludeInternalTopics Whether or not to include internal topics + * @param zkClient zkclient + */ +class SymmetricAllocator(group: String, consumerId: String, excludeInternalTopics: Boolean, zkClient: ZkClient) + extends PartitionAllocator with Logging { + + def allocate() = allocate(Set(consumerId)) + + def allocate(forConsumers: Set[String]) = { + val partitionOwnershipDecision = collection.mutable.Map[TopicAndPartition, String]() + val ctx = new AllocatorContext(group, consumerId, excludeInternalTopics, zkClient) + + val consumers = ZkUtils.getConsumersInGroup(zkClient, group).sorted + + // Check that all instances have identical subscriptions. + require(ctx.myTopicCount.pattern == TopicCount.whiteListPattern || + ctx.myTopicCount.pattern == TopicCount.blackListPattern, + "Symmetric allocation is allowed only with whitelists or blacklists.") + consumers.foreach(otherConsumer => { + val otherTopicCount = TopicCount.constructTopicCount(group, otherConsumer, zkClient, excludeInternalTopics) + require(otherTopicCount.pattern == ctx.myTopicCount.pattern, + "Symmetric allocation is allowed only if all consumers have identical subscription patterns.\n" + + "Consumer %s has pattern %s and consumer %s has pattern %s." + .format(consumerId, ctx.myTopicCount.pattern, otherConsumer, otherTopicCount.pattern)) + require(otherTopicCount.getTopicCountMap == ctx.myTopicCount.getTopicCountMap, + "Symmetric allocation is allowed only if all consumers have identical topic subscriptions.\n" + + "Consumer %s has topic-count %s and consumer %s has topic-count %s." + .format(consumerId, ctx.myTopicCount.getTopicCountMap, otherConsumer, otherTopicCount.getTopicCountMap)) + }) + + // If subscriptions are identical then the threadIds will also be identical for each consumer. + val streamCount = ctx.myTopicThreadIdsMap.head._2.size + + /* + * Generate an ordered (by consumerid-threadid) sequence of (consumerid-threadId, isLocal) pairs to do the + * assignment. We need to pair it with the consumer ID since we only need to return allocations for the consumers + * requested in allocate(forConsumers). + */ + val allThreadIds = consumers.flatMap(consumer => + (0 until streamCount).map(id => (TopicCount.makeThreadId(consumer, id), consumer))) // ordered by consumerid-threadid + + val threadAllocator = Utils.circularIterator(allThreadIds) + + val allTopicPartitions = ctx.partitionsPerTopicMap.flatMap { case(topic, partitions) => + info("Consumer %s using symmetric allocation to rebalance the following partitions: %s for topic %s with consumers: %s" + .format(consumerId, partitions, topic, consumers)) + partitions.map(partition => { + TopicAndPartition(topic, partition) + }) + }.toSeq.sortWith((topicPartition1, topicPartition2) => { + /** + * Sample ordering: topic0-0, topic1-0, ..., topic0-1, topic1-1, ... + * This helps reduce the likelihood of all partitions of a given topic ending up on one consumer (if it has a high + * enough stream count). + */ + if (topicPartition1.partition == topicPartition2.partition) + topicPartition1.topic < topicPartition2.topic + else + topicPartition1.partition < topicPartition2.partition + }) + + allTopicPartitions.foreach(topicPartition => { + val threadId = threadAllocator.next() + if (forConsumers.contains(threadId._2)) + partitionOwnershipDecision += (topicPartition -> threadId._1) + }) + + partitionOwnershipDecision + } +} + +/** + * Topic-level round-robin allocation strategy - while the global round-robin allocator works only with wildcards and + * identical subscriptions, this strategy accomodates asymmetric subscriptions (i.e., different stream counts) and a + * mix of wildcard and static subscriptions. It executes on a per-topic basis. For each topic, it lays out the available + * partitions and the available consumer threads and proceeds to do a round-robin assignment at two levels: first it + * selects the next available consumer instance and then it selects the next available consumer thread within that + * instance. + * + * @param group Consumer group + * @param consumerId Consumer id of the consumer in the group for which we are doing the allocation + * @param excludeInternalTopics Whether or not to include internal topics + * @param zkClient zkclient + */ +class RoundRobinAllocator(group: String, consumerId: String, excludeInternalTopics: Boolean, zkClient: ZkClient) + extends PartitionAllocator with Logging { + + def allocate() = allocate(Set(consumerId)) + + def allocate(forConsumers: Set[String]) = { + val partitionOwnershipDecision = collection.mutable.Map[TopicAndPartition, String]() + val ctx = new AllocatorContext(group, consumerId, excludeInternalTopics, zkClient) + + val consumers = ZkUtils.getConsumersInGroup(zkClient, group).sorted + + val consumersForTopic = mutable.Map[String, mutable.ListBuffer[String]]() + + val threadAllocatorsForConsumer = consumers.map(consumer => { + (consumer, { + val threadIdsByTopic = ZkUtils.getConsumerThreadIds(zkClient, group, consumer, excludeInternalTopics) + threadIdsByTopic.map { case (topic, threadIds) => + if (ctx.myTopicThreadIdsMap.contains(topic)) { + val consumersSoFar = consumersForTopic.getOrElse(topic, mutable.ListBuffer[String]()) + consumersForTopic.put(topic, consumersSoFar += consumer) + } + (topic, Utils.circularIterator(threadIds.toSeq.sorted)) + } + }) + }).toMap + + val consumerAllocatorsForTopic = consumersForTopic.map(e => (e._1, Utils.circularIterator(e._2))).toMap + + // now do the assignment + ctx.myTopicThreadIdsMap.keys.foreach(topic => { + val curPartitions = ctx.partitionsPerTopicMap.get(topic).get + info("Consumer %s using round-robin allocation to rebalance the following partitions: %s for topic %s with consumers: %s" + .format(consumerId, curPartitions, topic, consumers)) + curPartitions.foreach(partition => { + val assignedConsumer = consumerAllocatorsForTopic(topic).next() + val assignedThread = threadAllocatorsForConsumer(assignedConsumer)(topic).next() + if (forConsumers.contains(assignedConsumer)) + partitionOwnershipDecision += (TopicAndPartition(topic, partition) -> assignedThread) + }) + }) + + partitionOwnershipDecision + } +} + +class RangeAllocator(group: String, consumerId: String, excludeInternalTopics: Boolean, zkClient: ZkClient) + extends PartitionAllocator with Logging { + + def allocate() = { + val partitionOwnershipDecision = collection.mutable.Map[TopicAndPartition, String]() + val ctx = new AllocatorContext(group, consumerId, excludeInternalTopics, zkClient) + val consumersPerTopicMap = ZkUtils.getConsumersPerTopic(zkClient, group, excludeInternalTopics) + + for ((topic, consumerThreadIdSet) <- ctx.myTopicThreadIdsMap) { + val curConsumers = consumersPerTopicMap.get(topic).get + val curPartitions: Seq[Int] = ctx.partitionsPerTopicMap.get(topic).get + + val nPartsPerConsumer = curPartitions.size / curConsumers.size + val nConsumersWithExtraPart = curPartitions.size % curConsumers.size + + info("Consumer " + consumerId + " rebalancing the following partitions: " + curPartitions + + " for topic " + topic + " with consumers: " + curConsumers) + + for (consumerThreadId <- consumerThreadIdSet) { + val myConsumerPosition = curConsumers.indexOf(consumerThreadId) + assert(myConsumerPosition >= 0) + val startPart = nPartsPerConsumer*myConsumerPosition + myConsumerPosition.min(nConsumersWithExtraPart) + val nParts = nPartsPerConsumer + (if (myConsumerPosition + 1 > nConsumersWithExtraPart) 0 else 1) + + /** + * Range-partition the sorted partitions to consumers for better locality. + * The first few consumers pick up an extra partition, if any. + */ + if (nParts <= 0) + warn("No broker partitions consumed by consumer thread " + consumerThreadId + " for topic " + topic) + else { + for (i <- startPart until startPart + nParts) { + val partition = curPartitions(i) + info(consumerThreadId + " attempting to claim partition " + partition) + // record the partition ownership decision + partitionOwnershipDecision += (TopicAndPartition(topic, partition) -> consumerThreadId) + } + } + } + } + + partitionOwnershipDecision + } +} diff --git a/core/src/main/scala/kafka/consumer/TopicCount.scala b/core/src/main/scala/kafka/consumer/TopicCount.scala index c793110..2292b79 100644 --- a/core/src/main/scala/kafka/consumer/TopicCount.scala +++ b/core/src/main/scala/kafka/consumer/TopicCount.scala @@ -27,25 +27,28 @@ private[kafka] trait TopicCount { def getConsumerThreadIdsPerTopic: Map[String, Set[String]] def getTopicCountMap: Map[String, Int] def pattern: String - - protected def makeConsumerThreadIdsPerTopic(consumerIdString: String, - topicCountMap: Map[String, Int]) = { + +} + +private[kafka] object TopicCount extends Logging { + val whiteListPattern = "white_list" + val blackListPattern = "black_list" + val staticPattern = "static" + + def makeThreadId(consumerIdString: String, threadId: Int) = consumerIdString + "-" + threadId + + def makeConsumerThreadIdsPerTopic(consumerIdString: String, + topicCountMap: Map[String, Int]) = { val consumerThreadIdsPerTopicMap = new mutable.HashMap[String, Set[String]]() for ((topic, nConsumers) <- topicCountMap) { val consumerSet = new mutable.HashSet[String] assert(nConsumers >= 1) for (i <- 0 until nConsumers) - consumerSet += consumerIdString + "-" + i + consumerSet += makeThreadId(consumerIdString, i) consumerThreadIdsPerTopicMap.put(topic, consumerSet) } consumerThreadIdsPerTopicMap } -} - -private[kafka] object TopicCount extends Logging { - val whiteListPattern = "white_list" - val blackListPattern = "black_list" - val staticPattern = "static" def constructTopicCount(group: String, consumerId: String, zkClient: ZkClient, excludeInternalTopics: Boolean) : TopicCount = { val dirs = new ZKGroupDirs(group) @@ -101,7 +104,7 @@ private[kafka] class StaticTopicCount(val consumerIdString: String, val topicCountMap: Map[String, Int]) extends TopicCount { - def getConsumerThreadIdsPerTopic = makeConsumerThreadIdsPerTopic(consumerIdString, topicCountMap) + def getConsumerThreadIdsPerTopic = TopicCount.makeConsumerThreadIdsPerTopic(consumerIdString, topicCountMap) override def equals(obj: Any): Boolean = { obj match { @@ -124,7 +127,7 @@ private[kafka] class WildcardTopicCount(zkClient: ZkClient, def getConsumerThreadIdsPerTopic = { val wildcardTopics = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerTopicsPath) .filter(topic => topicFilter.isTopicAllowed(topic, excludeInternalTopics)) - makeConsumerThreadIdsPerTopic(consumerIdString, Map(wildcardTopics.map((_, numStreams)): _*)) + TopicCount.makeConsumerThreadIdsPerTopic(consumerIdString, Map(wildcardTopics.map((_, numStreams)): _*)) } def getTopicCountMap = Map(topicFilter.regex -> numStreams) diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 65f518d..0c2a628 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -513,9 +513,22 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, class ZKRebalancerListener(val group: String, val consumerIdString: String, val kafkaMessageAndMetadataStreams: mutable.Map[String,List[KafkaStream[_,_]]]) extends IZkChildListener { + private val partitionAllocator = config.partitionDistributionStrategy match { + case "roundrobin" => new RoundRobinAllocator(group, consumerIdString, config.excludeInternalTopics, zkClient) + case "symmetric" => new SymmetricAllocator(group, consumerIdString, config.excludeInternalTopics, zkClient) + case _ => new RangeAllocator(group, consumerIdString, config.excludeInternalTopics, zkClient) + } private var isWatcherTriggered = false private val lock = new ReentrantLock private val cond = lock.newCondition() + + @volatile private var allTopicsOwnedPartitionsCount = 0 + newGauge("AllTopics-OwnedPartitionsCount", new Gauge[Int] { + def value() = allTopicsOwnedPartitionsCount + }) + + private def ownedPartitionsCountMetricName(topic: String) = "%s-OwnedPartitionsCount".format(topic) + private val watcherExecutorThread = new Thread(consumerIdString + "_watcher_executor") { override def run() { info("starting watcher executor thread for consumer " + consumerIdString) @@ -564,10 +577,13 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private def releasePartitionOwnership(localTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]])= { info("Releasing partition ownership") for ((topic, infos) <- localTopicRegistry) { - for(partition <- infos.keys) + for(partition <- infos.keys) { deletePartitionOwnershipFromZK(topic, partition) + } + removeMetric(ownedPartitionsCountMetricName(topic)) localTopicRegistry.remove(topic) } + allTopicsOwnedPartitionsCount = 0 } def resetState() { @@ -617,7 +633,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private def rebalance(cluster: Cluster): Boolean = { val myTopicThreadIdsMap = TopicCount.constructTopicCount( group, consumerIdString, zkClient, config.excludeInternalTopics).getConsumerThreadIdsPerTopic - val consumersPerTopicMap = getConsumersPerTopic(zkClient, group, config.excludeInternalTopics) val brokers = getAllBrokersInCluster(zkClient) if (brokers.size == 0) { // This can happen in a rare case when there are no brokers available in the cluster when the consumer is started. @@ -628,9 +643,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, true } else { - val partitionsAssignmentPerTopicMap = getPartitionAssignmentForTopics(zkClient, myTopicThreadIdsMap.keySet.toSeq) - val partitionsPerTopicMap = partitionsAssignmentPerTopicMap.map(p => (p._1, p._2.keySet.toSeq.sorted)) - /** * fetchers must be stopped to avoid data duplication, since if the current * rebalancing attempt fails, the partitions that are released could be owned by another consumer. @@ -641,67 +653,39 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, releasePartitionOwnership(topicRegistry) - var partitionOwnershipDecision = new collection.mutable.HashMap[TopicAndPartition, String]() - val currentTopicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]] - - for ((topic, consumerThreadIdSet) <- myTopicThreadIdsMap) { - currentTopicRegistry.put(topic, new Pool[Int, PartitionTopicInfo]) - - val curConsumers = consumersPerTopicMap.get(topic).get - val curPartitions: Seq[Int] = partitionsPerTopicMap.get(topic).get - - val nPartsPerConsumer = curPartitions.size / curConsumers.size - val nConsumersWithExtraPart = curPartitions.size % curConsumers.size - - info("Consumer " + consumerIdString + " rebalancing the following partitions: " + curPartitions + - " for topic " + topic + " with consumers: " + curConsumers) - - for (consumerThreadId <- consumerThreadIdSet) { - val myConsumerPosition = curConsumers.indexOf(consumerThreadId) - assert(myConsumerPosition >= 0) - val startPart = nPartsPerConsumer*myConsumerPosition + myConsumerPosition.min(nConsumersWithExtraPart) - val nParts = nPartsPerConsumer + (if (myConsumerPosition + 1 > nConsumersWithExtraPart) 0 else 1) - - /** - * Range-partition the sorted partitions to consumers for better locality. - * The first few consumers pick up an extra partition, if any. - */ - if (nParts <= 0) - warn("No broker partitions consumed by consumer thread " + consumerThreadId + " for topic " + topic) - else { - for (i <- startPart until startPart + nParts) { - val partition = curPartitions(i) - info(consumerThreadId + " attempting to claim partition " + partition) - // record the partition ownership decision - partitionOwnershipDecision += (TopicAndPartition(topic, partition) -> consumerThreadId) - } - } - } - } + val partitionOwnershipDecision = partitionAllocator.allocate() + val currentTopicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]]( + valueFactory = Some((topic: String) => new Pool[Int, PartitionTopicInfo])) // fetch current offsets for all topic-partitions val topicPartitions = partitionOwnershipDecision.keySet.toSeq + val offsetFetchResponseOpt = fetchOffsets(topicPartitions) if (isShuttingDown.get || !offsetFetchResponseOpt.isDefined) false else { val offsetFetchResponse = offsetFetchResponseOpt.get - topicPartitions.foreach { topicAndPartition => + topicPartitions.foreach(topicAndPartition => { val (topic, partition) = topicAndPartition.asTuple val offset = offsetFetchResponse.requestInfo(topicAndPartition).offset val threadId = partitionOwnershipDecision(topicAndPartition) addPartitionTopicInfo(currentTopicRegistry, partition, topic, offset, threadId) - } + }) /** * move the partition ownership here, since that can be used to indicate a truly successful rebalancing attempt * A rebalancing attempt is completed successfully only after the fetchers have been started correctly */ - if(reflectPartitionOwnershipDecision(partitionOwnershipDecision.toMap)) { - info("Updating the cache") - debug("Partitions per topic cache " + partitionsPerTopicMap) - debug("Consumers per topic cache " + consumersPerTopicMap) + if(reflectPartitionOwnershipDecision(partitionOwnershipDecision)) { + allTopicsOwnedPartitionsCount = partitionOwnershipDecision.size + + partitionOwnershipDecision.view.groupBy(_._1.topic).foreach { case (topic, partitionThreadPairs) => + newGauge(ownedPartitionsCountMetricName(topic), new Gauge[Int] { + def value() = partitionThreadPairs.size + }) + } + topicRegistry = currentTopicRegistry updateFetcher(cluster) true @@ -808,7 +792,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private def addPartitionTopicInfo(currentTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]], partition: Int, topic: String, offset: Long, consumerThreadId: String) { - val partTopicInfoMap = currentTopicRegistry.get(topic) + val partTopicInfoMap = currentTopicRegistry.getAndMaybePut(topic) val queue = topicThreadIdAndQueues.get((topic, consumerThreadId)) val consumedOffset = new AtomicLong(offset) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index ecbfa0f..8763968 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -133,9 +133,9 @@ class RequestSendThread(val controllerId: Int, isSendSuccessful = true } catch { case e: Throwable => // if the send was not successful, reconnect to broker and resend the message - error(("Controller %d epoch %d failed to send request %s to broker %s. " + + error(("Controller %d epoch %d failed to send %s request with correlation id %s to broker %s. " + "Reconnecting to broker.").format(controllerId, controllerContext.epoch, - request.toString, toBroker.toString()), e) + RequestKeys.nameForKey(request.requestId.get), request.correlationId, toBroker.toString()), e) channel.disconnect() connectToBroker(toBroker, channel) isSendSuccessful = false @@ -153,8 +153,8 @@ class RequestSendThread(val controllerId: Int, case RequestKeys.UpdateMetadataKey => response = UpdateMetadataResponse.readFrom(receive.buffer) } - stateChangeLogger.trace("Controller %d epoch %d received response %s for a request sent to broker %s" - .format(controllerId, controllerContext.epoch, response.toString, toBroker.toString)) + stateChangeLogger.trace("Controller %d epoch %d received response correlationId %d for a request sent to broker %s" + .format(controllerId, controllerContext.epoch, response.correlationId, toBroker.toString())) if(callback != null) { callback(response) diff --git a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala index 27fc1eb..08dcc55 100644 --- a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala @@ -21,6 +21,7 @@ import kafka.common.{OffsetAndMetadata, TopicAndPartition} class OffsetCommitRequest(groupId: String, requestInfo: java.util.Map[TopicAndPartition, OffsetAndMetadata], + versionId: Short, correlationId: Int, clientId: String) { val underlying = { @@ -32,6 +33,7 @@ class OffsetCommitRequest(groupId: String, kafka.api.OffsetCommitRequest( groupId = groupId, requestInfo = scalaMap, + versionId = versionId, correlationId = correlationId, clientId = clientId ) diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala index b0b7be1..7e6da16 100644 --- a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala @@ -24,10 +24,10 @@ import kafka.common.ErrorMapping import kafka.network.RequestChannel.Response class TopicMetadataRequest(val versionId: Short, - val correlationId: Int, + override val correlationId: Int, val clientId: String, val topics: java.util.List[String]) - extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey)) { + extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey), correlationId) { val underlying: kafka.api.TopicMetadataRequest = { import scala.collection.JavaConversions._ diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala index a20ab90..0dbcb85 100644 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala @@ -51,4 +51,8 @@ trait KafkaMetricsGroup extends Logging { def newTimer(name: String, durationUnit: TimeUnit, rateUnit: TimeUnit) = Metrics.defaultRegistry().newTimer(metricName(name), durationUnit, rateUnit) + def removeMetric(name: String) { + Metrics.defaultRegistry().removeMetric(metricName(name)) + } + } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index fd5f12e..0b668f2 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -591,9 +591,8 @@ class KafkaApis(val requestChannel: RequestChannel, def handleTopicMetadataRequest(request: RequestChannel.Request) { val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet) - val brokers = metadataCache.getAliveBrokers - trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(topicMetadata.mkString(","), brokers.mkString(","), metadataRequest.correlationId, metadataRequest.clientId)) - val response = new TopicMetadataResponse(brokers, topicMetadata, metadataRequest.correlationId) + trace("Sending topic metadata %s for correlation id %d to client %s".format(topicMetadata.mkString(","), metadataRequest.correlationId, metadataRequest.clientId)) + val response = new TopicMetadataResponse(topicMetadata, metadataRequest.correlationId) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 7cd40e1..3198cdf 100644 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -81,12 +81,6 @@ private[server] class MetadataCache { topicResponses } - def getAliveBrokers = { - inLock(partitionMetadataLock.readLock()) { - aliveBrokers.values.toList - } - } - def addOrUpdatePartitionInfo(topic: String, partitionId: Int, stateInfo: PartitionStateInfo) { diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala index af47836..c040f49 100644 --- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala +++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala @@ -118,7 +118,7 @@ object ReplicaVerificationTool extends Logging { info("Getting topic metatdata...") val metadataTargetBrokers = ClientUtils.parseBrokerList(options.valueOf(brokerListOpt)) val topicsMetadataResponse = ClientUtils.fetchTopicMetadata(Set[String](), metadataTargetBrokers, clientId, maxWaitMs) - val brokerMap = topicsMetadataResponse.brokers.map(b => (b.id, b)).toMap + val brokerMap = topicsMetadataResponse.extractBrokers(topicsMetadataResponse.topicsMetadata) val filteredTopicMetadata = topicsMetadataResponse.topicsMetadata.filter( topicMetadata => if (topicWhiteListFiler.isTopicAllowed(topicMetadata.topic, excludeInternalTopics = false)) true diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index dcdc1ce..b82bb4e 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -658,6 +658,11 @@ object ZkUtils extends Logging { getChildren(zkClient, dirs.consumerRegistryDir) } + def getConsumerThreadIds(zkClient:ZkClient, group: String, consumer: String, excludeInternalTopics: Boolean) = { + val topicCount = TopicCount.constructTopicCount(group, consumer, zkClient, excludeInternalTopics) + topicCount.getConsumerThreadIdsPerTopic + } + def getConsumersPerTopic(zkClient: ZkClient, group: String, excludeInternalTopics: Boolean) : mutable.Map[String, List[String]] = { val dirs = new ZKGroupDirs(group) val consumers = getChildrenParentMayNotExist(zkClient, dirs.consumerRegistryDir) diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index 847a36b..a2117b3 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -23,14 +23,9 @@ import junit.framework.Assert._ import java.nio.ByteBuffer import kafka.message.{Message, ByteBufferMessageSet} import kafka.cluster.Broker -import kafka.common.{OffsetAndMetadata, ErrorMapping, OffsetMetadataAndError} -import kafka.utils.SystemTime -import org.apache.kafka.common.requests._ -import org.apache.kafka.common.protocol.ApiKeys -import scala.Some +import kafka.common.{OffsetAndMetadata, TopicAndPartition, ErrorMapping, OffsetMetadataAndError} import kafka.controller.LeaderIsrAndControllerEpoch -import kafka.common.TopicAndPartition -import org.apache.kafka.common.TopicPartition +import kafka.utils.SystemTime object SerializationTestUtils { @@ -148,26 +143,16 @@ object SerializationTestUtils { } def createTestTopicMetadataResponse: TopicMetadataResponse = { - new TopicMetadataResponse(brokers, Seq(topicmetaData1, topicmetaData2), 1) + new TopicMetadataResponse(Seq(topicmetaData1, topicmetaData2), 1) } - def createTestOffsetCommitRequestV1: OffsetCommitRequest = { + def createTestOffsetCommitRequest: OffsetCommitRequest = { new OffsetCommitRequest("group 1", collection.immutable.Map( TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="some metadata", timestamp=SystemTime.milliseconds), TopicAndPartition(topic1, 1) -> OffsetAndMetadata(offset=100L, metadata=OffsetAndMetadata.NoMetadata, timestamp=SystemTime.milliseconds) )) } - def createTestOffsetCommitRequestV0: OffsetCommitRequest = { - new OffsetCommitRequest( - versionId = 0, - groupId = "group 1", - requestInfo = collection.immutable.Map( - TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="some metadata", timestamp=SystemTime.milliseconds), - TopicAndPartition(topic1, 1) -> OffsetAndMetadata(offset=100L, metadata=OffsetAndMetadata.NoMetadata, timestamp=SystemTime.milliseconds) - )) - } - def createTestOffsetCommitResponse: OffsetCommitResponse = { new OffsetCommitResponse(collection.immutable.Map(TopicAndPartition(topic1, 0) -> ErrorMapping.NoError, TopicAndPartition(topic1, 1) -> ErrorMapping.NoError)) @@ -195,31 +180,6 @@ object SerializationTestUtils { ConsumerMetadataResponse(Some(brokers.head), ErrorMapping.NoError) } - def createHeartbeatRequestAndHeader: HeartbeatRequestAndHeader = { - val header = new RequestHeader(ApiKeys.HEARTBEAT.id, 0.asInstanceOf[Short], "", 1) - val body = new HeartbeatRequest("group1", 1, "consumer1") - HeartbeatRequestAndHeader(header, body) - } - - def createHeartbeatResponseAndHeader: HeartbeatResponseAndHeader = { - val header = new ResponseHeader(1) - val body = new HeartbeatResponse(0.asInstanceOf[Short]) - HeartbeatResponseAndHeader(header, body) - } - - def createJoinGroupRequestAndHeader: JoinGroupRequestAndHeader = { - import scala.collection.JavaConversions._ - val header = new RequestHeader(ApiKeys.JOIN_GROUP.id, 0.asInstanceOf[Short], "", 1) - val body = new JoinGroupRequest("group1", 30000, List("topic1"), "consumer1", "strategy1"); - JoinGroupRequestAndHeader(header, body) - } - - def createJoinGroupResponseAndHeader: JoinGroupResponseAndHeader = { - import scala.collection.JavaConversions._ - val header = new ResponseHeader(1) - val body = new JoinGroupResponse(0.asInstanceOf[Short], 1, "consumer1", List(new TopicPartition("test11", 1))) - JoinGroupResponseAndHeader(header, body) - } } class RequestResponseSerializationTest extends JUnitSuite { @@ -234,31 +194,27 @@ class RequestResponseSerializationTest extends JUnitSuite { private val offsetResponse = SerializationTestUtils.createTestOffsetResponse private val topicMetadataRequest = SerializationTestUtils.createTestTopicMetadataRequest private val topicMetadataResponse = SerializationTestUtils.createTestTopicMetadataResponse - private val offsetCommitRequestV0 = SerializationTestUtils.createTestOffsetCommitRequestV0 - private val offsetCommitRequestV1 = SerializationTestUtils.createTestOffsetCommitRequestV1 + private val offsetCommitRequest = SerializationTestUtils.createTestOffsetCommitRequest private val offsetCommitResponse = SerializationTestUtils.createTestOffsetCommitResponse private val offsetFetchRequest = SerializationTestUtils.createTestOffsetFetchRequest private val offsetFetchResponse = SerializationTestUtils.createTestOffsetFetchResponse private val consumerMetadataRequest = SerializationTestUtils.createConsumerMetadataRequest private val consumerMetadataResponse = SerializationTestUtils.createConsumerMetadataResponse private val consumerMetadataResponseNoCoordinator = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode) - private val heartbeatRequest = SerializationTestUtils.createHeartbeatRequestAndHeader - private val heartbeatResponse = SerializationTestUtils.createHeartbeatResponseAndHeader - private val joinGroupRequest = SerializationTestUtils.createJoinGroupRequestAndHeader - private val joinGroupResponse = SerializationTestUtils.createJoinGroupResponseAndHeader @Test def testSerializationAndDeserialization() { val requestsAndResponses = - collection.immutable.Seq(leaderAndIsrRequest, leaderAndIsrResponse, stopReplicaRequest, - stopReplicaResponse, producerRequest, producerResponse, - fetchRequest, offsetRequest, offsetResponse, topicMetadataRequest, - topicMetadataResponse, offsetCommitRequestV0, offsetCommitRequestV1, - offsetCommitResponse, offsetFetchRequest, offsetFetchResponse, - consumerMetadataRequest, consumerMetadataResponse, - consumerMetadataResponseNoCoordinator, heartbeatRequest, - heartbeatResponse, joinGroupRequest, joinGroupResponse) + collection.immutable.Seq(leaderAndIsrRequest, leaderAndIsrResponse, + stopReplicaRequest, stopReplicaResponse, + producerRequest, producerResponse, + fetchRequest, + offsetRequest, offsetResponse, + topicMetadataRequest, topicMetadataResponse, + offsetCommitRequest, offsetCommitResponse, + offsetFetchRequest, offsetFetchResponse, + consumerMetadataRequest, consumerMetadataResponse, consumerMetadataResponseNoCoordinator) requestsAndResponses.foreach { original => val buffer = ByteBuffer.allocate(original.sizeInBytes) @@ -266,9 +222,7 @@ class RequestResponseSerializationTest extends JUnitSuite { buffer.rewind() val deserializer = original.getClass.getDeclaredMethod("readFrom", classOf[ByteBuffer]) val deserialized = deserializer.invoke(null, buffer) - assertFalse("All serialized bytes in " + original.getClass.getSimpleName + " should have been consumed", - buffer.hasRemaining) - assertEquals("The original and deserialized for " + original.getClass.getSimpleName + " should be the same.", original, deserialized) + assertEquals("The original and deserialized request/response should be the same.", original, deserialized) } } } diff --git a/core/src/test/scala/unit/kafka/consumer/PartitionAllocatorTest.scala b/core/src/test/scala/unit/kafka/consumer/PartitionAllocatorTest.scala new file mode 100644 index 0000000..b0369c8 --- /dev/null +++ b/core/src/test/scala/unit/kafka/consumer/PartitionAllocatorTest.scala @@ -0,0 +1,293 @@ +package unit.kafka.consumer + +import org.scalatest.junit.JUnit3Suite +import org.easymock.EasyMock +import org.I0Itec.zkclient.ZkClient +import org.apache.zookeeper.data.Stat +import kafka.consumer.{SymmetricAllocator, RoundRobinAllocator} +import kafka.utils.{TestUtils, Logging, ZkUtils, Json} +import unit.kafka.consumer.PartitionAllocatorTest.{StaticSubscriptionInfo, Scenario, WildcardSubscriptionInfo} +import junit.framework.Assert._ +import kafka.common.TopicAndPartition + +class PartitionAllocatorTest extends JUnit3Suite with Logging { + + def testSymmetricAllocator() { + (1 to PartitionAllocatorTest.TestCaseCount).foreach (testCase => { + val consumerCount = 2.max(TestUtils.random.nextInt(PartitionAllocatorTest.MaxConsumerCount + 1)) + val allConsumerIds = (1 to consumerCount).map("g1c" + _).toSet + val streamCount = 1.max(TestUtils.random.nextInt(PartitionAllocatorTest.MaxStreamCount + 1)) + val topicCount = PartitionAllocatorTest.MinTopicCount.max(TestUtils.random.nextInt(PartitionAllocatorTest.MaxTopicCount + 1)) + + val topicPartitionCounts = Map((1 to topicCount).map(topic => { + ("topic-" + topic, PartitionAllocatorTest.MinPartitionCount.max(TestUtils.random.nextInt(PartitionAllocatorTest.MaxPartitionCount + 1))) + }).toSeq:_*) + + val subscriptions = Map((1 to consumerCount).map(consumer => { + ("g1c" + consumer, WildcardSubscriptionInfo(streamCount, ".*", isWhitelist = true)) + }).toSeq:_*) + val scenario = Scenario("g1", topicPartitionCounts, subscriptions) + val zkClient = PartitionAllocatorTest.setupZkClientMock(scenario) + EasyMock.replay(zkClient) + + // to check that another (random) consumer's computed global allocation is identical to c1's computed global allocation + var cx = 1 + while (cx == 1) cx = 1.max(TestUtils.random.nextInt(consumerCount + 1)) + + val c1Allocation = new SymmetricAllocator("g1", "g1c1", excludeInternalTopics = true, zkClient) + .allocate(allConsumerIds) + PartitionAllocatorTest.checkAllocationsAreCompleteAndUnique(scenario, c1Allocation) + PartitionAllocatorTest.checkAllocationsAreUniform(scenario, streamCount, c1Allocation) + + val cxAllocation = new SymmetricAllocator("g1", "g1c" + cx, excludeInternalTopics = true, zkClient) + .allocate(allConsumerIds) + assertTrue("Scenario %s: inconsistent allocations between consumer 1 and %d.".format(scenario, cx), + cxAllocation == c1Allocation) + }) + } + + def testRoundRobinPartitionAllocator() { + /** + * Fully check (by hand) two asymmetric scenarios - one with only wildcard subscriptions and one with a mix of + * wildcard and static subscriptions. + * + * Automatically test a range of symmetric and asymmetric scenarios - check for coverage and uniqueness. + */ + + /** only wildcard scenario: c1 with two streams, c2 with three streams */ + val wildcardScenario = Scenario(group = "g1", + topicPartitionCounts = Map("x" -> 8, "y" -> 4, "z" -> 6), + subscriptions = Map("g1c1" -> WildcardSubscriptionInfo(streamCount = 2, + regex = ".*", + isWhitelist = true), + "g1c2" -> WildcardSubscriptionInfo(streamCount = 3, + regex = ".*", + isWhitelist = true))) + val allConsumerIds: Set[String] = Set("g1c1", "g1c2") + val wildcardScenarioZkClient = PartitionAllocatorTest.setupZkClientMock(wildcardScenario) + EasyMock.replay(wildcardScenarioZkClient) + + val wildcardScenarioAllocator = new RoundRobinAllocator("g1", "g1c1", excludeInternalTopics = true, wildcardScenarioZkClient) + val wildcardScenarioAllocation = wildcardScenarioAllocator.allocate(allConsumerIds) + + PartitionAllocatorTest.checkAllocationsAreCompleteAndUnique(wildcardScenario, wildcardScenarioAllocation) + + val expectedWildcardScenarioAllocation = Map( + TopicAndPartition("x", 0) -> "g1c1-0", TopicAndPartition("x", 1) -> "g1c2-0", TopicAndPartition("x", 2) -> "g1c1-1", + TopicAndPartition("x", 3) -> "g1c2-1", TopicAndPartition("x", 4) -> "g1c1-0", TopicAndPartition("x", 5) -> "g1c2-2", + TopicAndPartition("x", 6) -> "g1c1-1", TopicAndPartition("x", 7) -> "g1c2-0", + TopicAndPartition("y", 0) -> "g1c1-0", TopicAndPartition("y", 1) -> "g1c2-0", TopicAndPartition("y", 2) -> "g1c1-1", + TopicAndPartition("y", 3) -> "g1c2-1", + TopicAndPartition("z", 0) -> "g1c1-0", TopicAndPartition("z", 1) -> "g1c2-0", TopicAndPartition("z", 2) -> "g1c1-1", + TopicAndPartition("z", 3) -> "g1c2-1", TopicAndPartition("z", 4) -> "g1c1-0", TopicAndPartition("z", 5) -> "g1c2-2" + ) + assertTrue("Scenario %s: incorrect allocation\n".format(wildcardScenario), + expectedWildcardScenarioAllocation == wildcardScenarioAllocation) + + /** mixed scenario - i.e., both static and wildcard consumers */ + val mixedScenario = Scenario(group = "g1", + topicPartitionCounts = Map("x" -> 8, "y" -> 4, "z" -> 6), + subscriptions = Map("g1c1" -> WildcardSubscriptionInfo(streamCount = 2, + regex = ".*", + isWhitelist = true), + "g1c2" -> StaticSubscriptionInfo(Map("x" -> 3, "y" -> 1)))) + val mixedScenarioZkClient = PartitionAllocatorTest.setupZkClientMock(mixedScenario) + EasyMock.replay(mixedScenarioZkClient) + + val mixedScenarioAllocator = new RoundRobinAllocator("g1", "g1c1", excludeInternalTopics = true, mixedScenarioZkClient) + val mixedScenarioAllocation = mixedScenarioAllocator.allocate(allConsumerIds) + + PartitionAllocatorTest.checkAllocationsAreCompleteAndUnique(mixedScenario, mixedScenarioAllocation) + + val expectedMixedScenarioAllocation = Map( + TopicAndPartition("x", 0) -> "g1c1-0", TopicAndPartition("x", 1) -> "g1c2-0", TopicAndPartition("x", 2) -> "g1c1-1", + TopicAndPartition("x", 3) -> "g1c2-1", TopicAndPartition("x", 4) -> "g1c1-0", TopicAndPartition("x", 5) -> "g1c2-2", + TopicAndPartition("x", 6) -> "g1c1-1", TopicAndPartition("x", 7) -> "g1c2-0", + TopicAndPartition("y", 0) -> "g1c1-0", TopicAndPartition("y", 1) -> "g1c2-0", TopicAndPartition("y", 2) -> "g1c1-1", + TopicAndPartition("y", 3) -> "g1c2-0", + TopicAndPartition("z", 0) -> "g1c1-0", TopicAndPartition("z", 1) -> "g1c1-1", TopicAndPartition("z", 2) -> "g1c1-0", + TopicAndPartition("z", 3) -> "g1c1-1", TopicAndPartition("z", 4) -> "g1c1-0", TopicAndPartition("z", 5) -> "g1c1-1" + ) + assertTrue("Scenario %s: incorrect allocation\n".format(mixedScenario), + expectedMixedScenarioAllocation == mixedScenarioAllocation) + + /** various scenarios with only wildcard consumers */ + (1 to PartitionAllocatorTest.TestCaseCount).foreach (testCase => { + val consumerCount = 2.max(TestUtils.random.nextInt(PartitionAllocatorTest.MaxConsumerCount + 1)) + val allConsumerIds = (1 to consumerCount).map("g1c" + _).toSet + val streamCount = 1.max(TestUtils.random.nextInt(PartitionAllocatorTest.MaxStreamCount + 1)) + val topicCount = PartitionAllocatorTest.MinTopicCount.max(TestUtils.random.nextInt(PartitionAllocatorTest.MaxTopicCount + 1)) + + val topicPartitionCounts = Map((1 to topicCount).map(topic => { + ("topic-" + topic, PartitionAllocatorTest.MinPartitionCount.max(TestUtils.random.nextInt(PartitionAllocatorTest.MaxPartitionCount))) + }).toSeq:_*) + + // to check that another random consumer's computed global allocation is identical to C1's computed global allocation + var cx = 1 + while (cx == 1) cx = 1.max(TestUtils.random.nextInt(consumerCount + 1)) + + val symmetricSubscriptions = Map((1 to consumerCount).map(consumer => { + ("g1c" + consumer, WildcardSubscriptionInfo(streamCount, ".*", isWhitelist = true)) + }).toSeq:_*) + val symmetricScenario = Scenario("g1", topicPartitionCounts, symmetricSubscriptions) + val symmetricZkClient = PartitionAllocatorTest.setupZkClientMock(symmetricScenario) + EasyMock.replay(symmetricZkClient) + val c1SymmetricAllocation = new RoundRobinAllocator("g1", "g1c1", excludeInternalTopics = true, symmetricZkClient) + .allocate(allConsumerIds) + PartitionAllocatorTest.checkAllocationsAreCompleteAndUnique(symmetricScenario, c1SymmetricAllocation) + + val cxSymmetricAllocation = new RoundRobinAllocator("g1", "g1c" + cx, excludeInternalTopics = true, symmetricZkClient) + .allocate(allConsumerIds) + assertTrue("Scenario %s: inconsistent allocations between consumer 1 and %d.".format(symmetricScenario, cx), + cxSymmetricAllocation == c1SymmetricAllocation) + + val asymmetricSubscriptions = Map((1 to consumerCount).map(consumer => { + val streamCount = 1.max(TestUtils.random.nextInt(1)) + ("g1c" + consumer, WildcardSubscriptionInfo(streamCount, ".*", isWhitelist = true)) + }).toSeq:_*) + val asymmetricScenario = Scenario("g1", topicPartitionCounts, asymmetricSubscriptions) + val asymmetricZkClient = PartitionAllocatorTest.setupZkClientMock(asymmetricScenario) + EasyMock.replay(asymmetricZkClient) + val asymmetricAllocation = new RoundRobinAllocator("g1", "g1c1", excludeInternalTopics = true, asymmetricZkClient) + .allocate(allConsumerIds) + PartitionAllocatorTest.checkAllocationsAreCompleteAndUnique(asymmetricScenario, asymmetricAllocation) + + val cxAsymmetricAllocation = new RoundRobinAllocator("g1", "g1c" + cx, excludeInternalTopics = true, asymmetricZkClient) + .allocate(allConsumerIds) + assertTrue("Scenario %s: inconsistent allocations between consumer 1 and %d.".format(asymmetricScenario, cx), + cxAsymmetricAllocation == asymmetricAllocation) + + }) + } + +} + +private object PartitionAllocatorTest extends Logging { + + private val TestCaseCount = 5 + private val MaxConsumerCount = 20 + private val MaxStreamCount = 10 + private val MaxTopicCount = 200 + private val MinTopicCount = 20 + private val MaxPartitionCount = 120 + private val MinPartitionCount = 8 + + private trait SubscriptionInfo { + def registrationString: String + } + + private case class StaticSubscriptionInfo(streamCounts: Map[String, Int]) extends SubscriptionInfo { + def registrationString = + Json.encode(Map("version" -> 1, + "subscription" -> streamCounts, + "pattern" -> "static", + "timestamp" -> 1234.toString)) + + override def toString = { + "Stream counts: " + streamCounts + } + } + + private case class WildcardSubscriptionInfo(streamCount: Int, regex: String, isWhitelist: Boolean) + extends SubscriptionInfo { + def registrationString = + Json.encode(Map("version" -> 1, + "subscription" -> Map(regex -> streamCount), + "pattern" -> (if (isWhitelist) "white_list" else "black_list"))) + + override def toString = { + "\"%s\":%d (%s)".format(regex, streamCount, if (isWhitelist) "whitelist" else "blacklist") + } + } + + private case class Scenario(group: String, + topicPartitionCounts: Map[String, Int], + /* consumerId -> SubscriptionInfo */ + subscriptions: Map[String, SubscriptionInfo]) { + override def toString = { + "\n" + + "Group : %s\n".format(group) + + "Topic partition counts : %s\n".format(topicPartitionCounts) + + "Consumer subscriptions : %s\n".format(subscriptions) + } + } + + private def setupZkClientMock(scenario: Scenario) = { + val consumers = java.util.Arrays.asList(scenario.subscriptions.keys.toSeq:_*) + + val zkClient = EasyMock.createStrictMock(classOf[ZkClient]) + EasyMock.checkOrder(zkClient, false) + + EasyMock.expect(zkClient.getChildren("/consumers/%s/ids".format(scenario.group))).andReturn(consumers) + EasyMock.expectLastCall().anyTimes() + + scenario.subscriptions.foreach { case(consumerId, subscriptionInfo) => + EasyMock.expect(zkClient.readData("/consumers/%s/ids/%s".format(scenario.group, consumerId), new Stat())) + .andReturn(subscriptionInfo.registrationString) + EasyMock.expectLastCall().anyTimes() + } + + scenario.topicPartitionCounts.foreach { case(topic, partitionCount) => + val replicaAssignment = Map((0 until partitionCount).map(partition => (partition.toString, Seq(0))):_*) + EasyMock.expect(zkClient.readData("/brokers/topics/%s".format(topic), new Stat())) + .andReturn(ZkUtils.replicaAssignmentZkData(replicaAssignment)) + EasyMock.expectLastCall().anyTimes() + } + + EasyMock.expect(zkClient.getChildren("/brokers/topics")).andReturn( + java.util.Arrays.asList(scenario.topicPartitionCounts.keys.toSeq:_*)) + EasyMock.expectLastCall().anyTimes() + + zkClient + } + + private def checkAllocationsAreCompleteAndUnique(scenario: Scenario, allocation: collection.Map[TopicAndPartition, String]) { + val allocatedPartitions = allocation.keySet + val givenPartitions = scenario.topicPartitionCounts.flatMap{ case (topic, partitionCount) => + (0 until partitionCount).map(partition => TopicAndPartition(topic, partition)) + }.toSet + + assertTrue("Scenario %s: the list of given partitions and assigned partitions are different.".format(scenario), + givenPartitions == allocatedPartitions) + + val counts = partitionOwnerCounts(allocation) + counts.foreach { case (topicPartition, count) => + assertTrue("Scenario %s: partition %s is owned by %d (i.e., more than one) consumer streams." + .format(scenario, topicPartition, count), count <= 1) + } + } + + private def checkAllocationsAreUniform(scenario: Scenario, streamCount: Int, allocation: collection.Map[TopicAndPartition, String]) { + val expectedMinOwnedCount = + scenario.topicPartitionCounts.valuesIterator.sum / (scenario.subscriptions.size * streamCount) + val expectedMaxOwnedCount = expectedMinOwnedCount + 1 + val validCounts = Seq(expectedMinOwnedCount, expectedMaxOwnedCount) + val actualCounts = PartitionAllocatorTest.partitionsOwnedCounts(allocation) + actualCounts.foreach { case(stream, count) => + assertTrue("Scenario %s: consumer stream %s owns %d partitions - expected range is [%d, %d]." + .format(scenario, stream, count, expectedMinOwnedCount, expectedMaxOwnedCount), validCounts.contains(count)) + } + } + + /** For each partition, count the number of consumers that own that partition (should be exactly one). */ + private def partitionOwnerCounts(allocation: collection.Map[TopicAndPartition, String]) = { + val ownerCounts = collection.mutable.Map[TopicAndPartition, Int]() + allocation.foreach { case (topicPartition, owner) => + val updatedCount = ownerCounts.getOrElse(topicPartition, 0) + 1 + ownerCounts.put(topicPartition, updatedCount) + } + ownerCounts + } + + /** For each consumer stream, count the number of partitions that it owns. */ + private def partitionsOwnedCounts(allocation: collection.Map[TopicAndPartition, String]) = { + val ownedCounts = collection.mutable.Map[String, Int]() + allocation.foreach { case (topicPartition, owner) => + val updatedCount = ownedCounts.getOrElse(owner, 0) + 1 + ownedCounts.put(owner, updatedCount) + } + ownedCounts + } + +} +