diff --git a/aiokafka/admin/client.py b/aiokafka/admin/client.py index 32309d7b..cb436a47 100644 --- a/aiokafka/admin/client.py +++ b/aiokafka/admin/client.py @@ -4,10 +4,13 @@ from ssl import SSLContext from typing import List, Optional, Dict, Tuple, Any -from kafka.protocol.api import Request, Response -from kafka.protocol.metadata import MetadataRequest -from kafka.protocol.commit import OffsetFetchRequest, GroupCoordinatorRequest -from kafka.protocol.admin import ( +from aiokafka import __version__ +from aiokafka.client import AIOKafkaClient +from aiokafka.errors import IncompatibleBrokerVersion, for_code +from aiokafka.protocol.api import Request, Response +from aiokafka.protocol.metadata import MetadataRequest +from aiokafka.protocol.commit import OffsetFetchRequest, GroupCoordinatorRequest +from aiokafka.protocol.admin import ( CreatePartitionsRequest, CreateTopicsRequest, DeleteTopicsRequest, @@ -16,10 +19,6 @@ AlterConfigsRequest, ListGroupsRequest, ApiVersionRequest_v0) - -from aiokafka import __version__ -from aiokafka.client import AIOKafkaClient -from aiokafka.errors import IncompatibleBrokerVersion, for_code from aiokafka.structs import TopicPartition, OffsetAndMetadata from .config_resource import ConfigResourceType, ConfigResource @@ -149,7 +148,7 @@ def _matching_api_version(self, operation: List[Request]) -> int: supported by the broker. :param operation: A list of protocol operation versions from - kafka.protocol. + aiokafka.protocol. :return: The max matching version number between client and broker. """ api_key = operation[0].API_KEY diff --git a/aiokafka/client.py b/aiokafka/client.py index 371e4668..9ecd23cd 100644 --- a/aiokafka/client.py +++ b/aiokafka/client.py @@ -3,18 +3,17 @@ import random import time -from kafka.protocol.admin import DescribeAclsRequest_v2 -from kafka.protocol.commit import OffsetFetchRequest -from kafka.protocol.fetch import FetchRequest -from kafka.protocol.metadata import MetadataRequest -from kafka.protocol.offset import OffsetRequest -from kafka.protocol.produce import ProduceRequest - import aiokafka.errors as Errors from aiokafka import __version__ from aiokafka.conn import collect_hosts, create_conn, CloseReason from aiokafka.cluster import ClusterMetadata +from aiokafka.protocol.admin import DescribeAclsRequest_v2 +from aiokafka.protocol.commit import OffsetFetchRequest from aiokafka.protocol.coordination import FindCoordinatorRequest +from aiokafka.protocol.fetch import FetchRequest +from aiokafka.protocol.metadata import MetadataRequest +from aiokafka.protocol.offset import OffsetRequest +from aiokafka.protocol.produce import ProduceRequest from aiokafka.errors import ( KafkaError, KafkaConnectionError, @@ -525,11 +524,11 @@ async def check_version(self, node_id=None): assert self.cluster.brokers(), 'no brokers in metadata' node_id = list(self.cluster.brokers())[0].nodeId - from kafka.protocol.admin import ( + from aiokafka.protocol.admin import ( ListGroupsRequest_v0, ApiVersionRequest_v0) - from kafka.protocol.commit import ( + from aiokafka.protocol.commit import ( OffsetFetchRequest_v0, GroupCoordinatorRequest_v0) - from kafka.protocol.metadata import MetadataRequest_v0 + from aiokafka.protocol.metadata import MetadataRequest_v0 test_cases = [ ((0, 10), ApiVersionRequest_v0()), ((0, 9), ListGroupsRequest_v0()), diff --git a/aiokafka/conn.py b/aiokafka/conn.py index 2be93dc3..da27fd27 100644 --- a/aiokafka/conn.py +++ b/aiokafka/conn.py @@ -16,18 +16,17 @@ import weakref import async_timeout -from kafka.protocol.api import RequestHeader -from kafka.protocol.admin import ( + +import aiokafka.errors as Errors +from aiokafka.abc import AbstractTokenProvider +from aiokafka.protocol.api import RequestHeader +from aiokafka.protocol.admin import ( SaslHandShakeRequest, SaslAuthenticateRequest, ApiVersionRequest ) -from kafka.protocol.commit import ( +from aiokafka.protocol.commit import ( GroupCoordinatorResponse_v0 as GroupCoordinatorResponse) - -import aiokafka.errors as Errors from aiokafka.util import create_future, create_task, get_running_loop, wait_for -from aiokafka.abc import AbstractTokenProvider - try: import gssapi except ImportError: diff --git a/aiokafka/consumer/fetcher.py b/aiokafka/consumer/fetcher.py index 2a3394b3..6d08bd21 100644 --- a/aiokafka/consumer/fetcher.py +++ b/aiokafka/consumer/fetcher.py @@ -6,12 +6,12 @@ from itertools import chain import async_timeout -from kafka.protocol.offset import OffsetRequest -from kafka.protocol.fetch import FetchRequest import aiokafka.errors as Errors from aiokafka.errors import ( ConsumerStoppedError, RecordTooLargeError, KafkaTimeoutError) +from aiokafka.protocol.offset import OffsetRequest +from aiokafka.protocol.fetch import FetchRequest from aiokafka.record.memory_records import MemoryRecords from aiokafka.record.control_record import ControlRecord, ABORT_MARKER from aiokafka.structs import OffsetAndTimestamp, TopicPartition, ConsumerRecord diff --git a/aiokafka/consumer/group_coordinator.py b/aiokafka/consumer/group_coordinator.py index 8a8c76f4..963d558c 100644 --- a/aiokafka/consumer/group_coordinator.py +++ b/aiokafka/consumer/group_coordinator.py @@ -4,17 +4,16 @@ import copy import time -from kafka.protocol.commit import ( - OffsetCommitRequest_v2 as OffsetCommitRequest, - OffsetFetchRequest_v1 as OffsetFetchRequest) -from kafka.protocol.group import ( - HeartbeatRequest, JoinGroupRequest, LeaveGroupRequest, SyncGroupRequest) - import aiokafka.errors as Errors -from aiokafka.structs import OffsetAndMetadata, TopicPartition from aiokafka.client import ConnectionGroup, CoordinationType from aiokafka.coordinator.assignors.roundrobin import RoundRobinPartitionAssignor from aiokafka.coordinator.protocol import ConsumerProtocol +from aiokafka.protocol.commit import ( + OffsetCommitRequest_v2 as OffsetCommitRequest, + OffsetFetchRequest_v1 as OffsetFetchRequest) +from aiokafka.protocol.group import ( + HeartbeatRequest, JoinGroupRequest, LeaveGroupRequest, SyncGroupRequest) +from aiokafka.structs import OffsetAndMetadata, TopicPartition from aiokafka.util import create_future, create_task log = logging.getLogger(__name__) diff --git a/aiokafka/coordinator/assignors/sticky/sticky_assignor.py b/aiokafka/coordinator/assignors/sticky/sticky_assignor.py index 05e14ef2..ae2235f5 100644 --- a/aiokafka/coordinator/assignors/sticky/sticky_assignor.py +++ b/aiokafka/coordinator/assignors/sticky/sticky_assignor.py @@ -2,9 +2,6 @@ from collections import defaultdict, namedtuple from copy import deepcopy -from kafka.protocol.struct import Struct -from kafka.protocol.types import String, Array, Int32 - from aiokafka.coordinator.assignors.abstract import AbstractPartitionAssignor from aiokafka.coordinator.assignors.sticky.partition_movements import PartitionMovements from aiokafka.coordinator.assignors.sticky.sorted_set import SortedSet @@ -13,6 +10,8 @@ ConsumerProtocolMemberAssignment, ) from aiokafka.coordinator.protocol import Schema +from aiokafka.protocol.struct import Struct +from aiokafka.protocol.types import String, Array, Int32 from aiokafka.structs import TopicPartition log = logging.getLogger(__name__) diff --git a/aiokafka/coordinator/base.py b/aiokafka/coordinator/base.py index 4489884d..ea6b4ccd 100644 --- a/aiokafka/coordinator/base.py +++ b/aiokafka/coordinator/base.py @@ -6,18 +6,18 @@ import weakref from kafka.future import Future -from kafka.protocol.commit import GroupCoordinatorRequest, OffsetCommitRequest -from kafka.protocol.group import ( + +from aiokafka import errors as Errors +from aiokafka.metrics import AnonMeasurable +from aiokafka.metrics.stats import Avg, Count, Max, Rate +from aiokafka.protocol.commit import GroupCoordinatorRequest, OffsetCommitRequest +from aiokafka.protocol.group import ( HeartbeatRequest, JoinGroupRequest, LeaveGroupRequest, SyncGroupRequest, ) -from aiokafka import errors as Errors -from aiokafka.metrics import AnonMeasurable -from aiokafka.metrics.stats import Avg, Count, Max, Rate - from .heartbeat import Heartbeat log = logging.getLogger("aiokafka.coordinator") diff --git a/aiokafka/coordinator/consumer.py b/aiokafka/coordinator/consumer.py index 2c7ebb4e..8f6cdaba 100644 --- a/aiokafka/coordinator/consumer.py +++ b/aiokafka/coordinator/consumer.py @@ -5,12 +5,12 @@ import time from kafka.future import Future -from kafka.protocol.commit import OffsetCommitRequest, OffsetFetchRequest from kafka.util import WeakMethod import aiokafka.errors as Errors from aiokafka.metrics import AnonMeasurable from aiokafka.metrics.stats import Avg, Count, Max, Rate +from aiokafka.protocol.commit import OffsetCommitRequest, OffsetFetchRequest from aiokafka.structs import OffsetAndMetadata, TopicPartition from .base import BaseCoordinator, Generation diff --git a/aiokafka/coordinator/protocol.py b/aiokafka/coordinator/protocol.py index 0dfbe7f9..aa86a7ff 100644 --- a/aiokafka/coordinator/protocol.py +++ b/aiokafka/coordinator/protocol.py @@ -1,6 +1,5 @@ -from kafka.protocol.struct import Struct -from kafka.protocol.types import Array, Bytes, Int16, Int32, Schema, String - +from aiokafka.protocol.struct import Struct +from aiokafka.protocol.types import Array, Bytes, Int16, Int32, Schema, String from aiokafka.structs import TopicPartition diff --git a/aiokafka/producer/producer.py b/aiokafka/producer/producer.py index f5a57b73..12a07e7f 100644 --- a/aiokafka/producer/producer.py +++ b/aiokafka/producer/producer.py @@ -128,7 +128,7 @@ class AIOKafkaProducer: brokers or partitions. Default: 300000 request_timeout_ms (int): Produce request timeout in milliseconds. As it's sent as part of - :class:`~kafka.protocol.produce.ProduceRequest` (it's a blocking + :class:`~aiokafka.protocol.produce.ProduceRequest` (it's a blocking call), maximum waiting time can be up to ``2 * request_timeout_ms``. Default: 40000. diff --git a/aiokafka/producer/sender.py b/aiokafka/producer/sender.py index b8faf1f6..bc6c8f4e 100644 --- a/aiokafka/producer/sender.py +++ b/aiokafka/producer/sender.py @@ -3,8 +3,6 @@ import logging import time -from kafka.protocol.produce import ProduceRequest - import aiokafka.errors as Errors from aiokafka.client import ConnectionGroup, CoordinationType from aiokafka.errors import ( @@ -16,6 +14,7 @@ OutOfOrderSequenceNumber, TopicAuthorizationFailedError, GroupAuthorizationFailedError, TransactionalIdAuthorizationFailed, OperationNotAttempted) +from aiokafka.protocol.produce import ProduceRequest from aiokafka.protocol.transaction import ( InitProducerIdRequest, AddPartitionsToTxnRequest, EndTxnRequest, AddOffsetsToTxnRequest, TxnOffsetCommitRequest diff --git a/aiokafka/protocol/__init__.py b/aiokafka/protocol/__init__.py index e69de29b..e001b571 100644 --- a/aiokafka/protocol/__init__.py +++ b/aiokafka/protocol/__init__.py @@ -0,0 +1,46 @@ +API_KEYS = { + 0: "Produce", + 1: "Fetch", + 2: "ListOffsets", + 3: "Metadata", + 4: "LeaderAndIsr", + 5: "StopReplica", + 6: "UpdateMetadata", + 7: "ControlledShutdown", + 8: "OffsetCommit", + 9: "OffsetFetch", + 10: "FindCoordinator", + 11: "JoinGroup", + 12: "Heartbeat", + 13: "LeaveGroup", + 14: "SyncGroup", + 15: "DescribeGroups", + 16: "ListGroups", + 17: "SaslHandshake", + 18: "ApiVersions", + 19: "CreateTopics", + 20: "DeleteTopics", + 21: "DeleteRecords", + 22: "InitProducerId", + 23: "OffsetForLeaderEpoch", + 24: "AddPartitionsToTxn", + 25: "AddOffsetsToTxn", + 26: "EndTxn", + 27: "WriteTxnMarkers", + 28: "TxnOffsetCommit", + 29: "DescribeAcls", + 30: "CreateAcls", + 31: "DeleteAcls", + 32: "DescribeConfigs", + 33: "AlterConfigs", + 36: "SaslAuthenticate", + 37: "CreatePartitions", + 38: "CreateDelegationToken", + 39: "RenewDelegationToken", + 40: "ExpireDelegationToken", + 41: "DescribeDelegationToken", + 42: "DeleteGroups", + 45: "AlterPartitionReassignments", + 46: "ListPartitionReassignments", + 48: "DescribeClientQuotas", +} diff --git a/kafka/protocol/abstract.py b/aiokafka/protocol/abstract.py similarity index 57% rename from kafka/protocol/abstract.py rename to aiokafka/protocol/abstract.py index 2de65c4b..b52a79ce 100644 --- a/kafka/protocol/abstract.py +++ b/aiokafka/protocol/abstract.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import abc @@ -7,11 +5,11 @@ class AbstractType(object): __metaclass__ = abc.ABCMeta @abc.abstractmethod - def encode(cls, value): # pylint: disable=no-self-argument + def encode(cls, value): # pylint: disable=no-self-argument pass @abc.abstractmethod - def decode(cls, data): # pylint: disable=no-self-argument + def decode(cls, data): # pylint: disable=no-self-argument pass @classmethod diff --git a/aiokafka/protocol/admin.py b/aiokafka/protocol/admin.py new file mode 100644 index 00000000..e1b0ffc4 --- /dev/null +++ b/aiokafka/protocol/admin.py @@ -0,0 +1,1278 @@ +from .api import Request, Response +from .types import ( + Array, + Boolean, + Bytes, + Int8, + Int16, + Int32, + Int64, + Schema, + String, + Float64, + CompactString, + CompactArray, + TaggedFields, +) + + +class ApiVersionResponse_v0(Response): + API_KEY = 18 + API_VERSION = 0 + SCHEMA = Schema( + ("error_code", Int16), + ( + "api_versions", + Array(("api_key", Int16), ("min_version", Int16), ("max_version", Int16)), + ), + ) + + +class ApiVersionResponse_v1(Response): + API_KEY = 18 + API_VERSION = 1 + SCHEMA = Schema( + ("error_code", Int16), + ( + "api_versions", + Array(("api_key", Int16), ("min_version", Int16), ("max_version", Int16)), + ), + ("throttle_time_ms", Int32), + ) + + +class ApiVersionResponse_v2(Response): + API_KEY = 18 + API_VERSION = 2 + SCHEMA = ApiVersionResponse_v1.SCHEMA + + +class ApiVersionRequest_v0(Request): + API_KEY = 18 + API_VERSION = 0 + RESPONSE_TYPE = ApiVersionResponse_v0 + SCHEMA = Schema() + + +class ApiVersionRequest_v1(Request): + API_KEY = 18 + API_VERSION = 1 + RESPONSE_TYPE = ApiVersionResponse_v1 + SCHEMA = ApiVersionRequest_v0.SCHEMA + + +class ApiVersionRequest_v2(Request): + API_KEY = 18 + API_VERSION = 2 + RESPONSE_TYPE = ApiVersionResponse_v1 + SCHEMA = ApiVersionRequest_v0.SCHEMA + + +ApiVersionRequest = [ + ApiVersionRequest_v0, + ApiVersionRequest_v1, + ApiVersionRequest_v2, +] +ApiVersionResponse = [ + ApiVersionResponse_v0, + ApiVersionResponse_v1, + ApiVersionResponse_v2, +] + + +class CreateTopicsResponse_v0(Response): + API_KEY = 19 + API_VERSION = 0 + SCHEMA = Schema( + ("topic_errors", Array(("topic", String("utf-8")), ("error_code", Int16))) + ) + + +class CreateTopicsResponse_v1(Response): + API_KEY = 19 + API_VERSION = 1 + SCHEMA = Schema( + ( + "topic_errors", + Array( + ("topic", String("utf-8")), + ("error_code", Int16), + ("error_message", String("utf-8")), + ), + ) + ) + + +class CreateTopicsResponse_v2(Response): + API_KEY = 19 + API_VERSION = 2 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "topic_errors", + Array( + ("topic", String("utf-8")), + ("error_code", Int16), + ("error_message", String("utf-8")), + ), + ), + ) + + +class CreateTopicsResponse_v3(Response): + API_KEY = 19 + API_VERSION = 3 + SCHEMA = CreateTopicsResponse_v2.SCHEMA + + +class CreateTopicsRequest_v0(Request): + API_KEY = 19 + API_VERSION = 0 + RESPONSE_TYPE = CreateTopicsResponse_v0 + SCHEMA = Schema( + ( + "create_topic_requests", + Array( + ("topic", String("utf-8")), + ("num_partitions", Int32), + ("replication_factor", Int16), + ( + "replica_assignment", + Array(("partition_id", Int32), ("replicas", Array(Int32))), + ), + ( + "configs", + Array( + ("config_key", String("utf-8")), + ("config_value", String("utf-8")), + ), + ), + ), + ), + ("timeout", Int32), + ) + + +class CreateTopicsRequest_v1(Request): + API_KEY = 19 + API_VERSION = 1 + RESPONSE_TYPE = CreateTopicsResponse_v1 + SCHEMA = Schema( + ( + "create_topic_requests", + Array( + ("topic", String("utf-8")), + ("num_partitions", Int32), + ("replication_factor", Int16), + ( + "replica_assignment", + Array(("partition_id", Int32), ("replicas", Array(Int32))), + ), + ( + "configs", + Array( + ("config_key", String("utf-8")), + ("config_value", String("utf-8")), + ), + ), + ), + ), + ("timeout", Int32), + ("validate_only", Boolean), + ) + + +class CreateTopicsRequest_v2(Request): + API_KEY = 19 + API_VERSION = 2 + RESPONSE_TYPE = CreateTopicsResponse_v2 + SCHEMA = CreateTopicsRequest_v1.SCHEMA + + +class CreateTopicsRequest_v3(Request): + API_KEY = 19 + API_VERSION = 3 + RESPONSE_TYPE = CreateTopicsResponse_v3 + SCHEMA = CreateTopicsRequest_v1.SCHEMA + + +CreateTopicsRequest = [ + CreateTopicsRequest_v0, + CreateTopicsRequest_v1, + CreateTopicsRequest_v2, + CreateTopicsRequest_v3, +] +CreateTopicsResponse = [ + CreateTopicsResponse_v0, + CreateTopicsResponse_v1, + CreateTopicsResponse_v2, + CreateTopicsResponse_v3, +] + + +class DeleteTopicsResponse_v0(Response): + API_KEY = 20 + API_VERSION = 0 + SCHEMA = Schema( + ("topic_error_codes", Array(("topic", String("utf-8")), ("error_code", Int16))) + ) + + +class DeleteTopicsResponse_v1(Response): + API_KEY = 20 + API_VERSION = 1 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ("topic_error_codes", Array(("topic", String("utf-8")), ("error_code", Int16))), + ) + + +class DeleteTopicsResponse_v2(Response): + API_KEY = 20 + API_VERSION = 2 + SCHEMA = DeleteTopicsResponse_v1.SCHEMA + + +class DeleteTopicsResponse_v3(Response): + API_KEY = 20 + API_VERSION = 3 + SCHEMA = DeleteTopicsResponse_v1.SCHEMA + + +class DeleteTopicsRequest_v0(Request): + API_KEY = 20 + API_VERSION = 0 + RESPONSE_TYPE = DeleteTopicsResponse_v0 + SCHEMA = Schema(("topics", Array(String("utf-8"))), ("timeout", Int32)) + + +class DeleteTopicsRequest_v1(Request): + API_KEY = 20 + API_VERSION = 1 + RESPONSE_TYPE = DeleteTopicsResponse_v1 + SCHEMA = DeleteTopicsRequest_v0.SCHEMA + + +class DeleteTopicsRequest_v2(Request): + API_KEY = 20 + API_VERSION = 2 + RESPONSE_TYPE = DeleteTopicsResponse_v2 + SCHEMA = DeleteTopicsRequest_v0.SCHEMA + + +class DeleteTopicsRequest_v3(Request): + API_KEY = 20 + API_VERSION = 3 + RESPONSE_TYPE = DeleteTopicsResponse_v3 + SCHEMA = DeleteTopicsRequest_v0.SCHEMA + + +DeleteTopicsRequest = [ + DeleteTopicsRequest_v0, + DeleteTopicsRequest_v1, + DeleteTopicsRequest_v2, + DeleteTopicsRequest_v3, +] +DeleteTopicsResponse = [ + DeleteTopicsResponse_v0, + DeleteTopicsResponse_v1, + DeleteTopicsResponse_v2, + DeleteTopicsResponse_v3, +] + + +class ListGroupsResponse_v0(Response): + API_KEY = 16 + API_VERSION = 0 + SCHEMA = Schema( + ("error_code", Int16), + ( + "groups", + Array(("group", String("utf-8")), ("protocol_type", String("utf-8"))), + ), + ) + + +class ListGroupsResponse_v1(Response): + API_KEY = 16 + API_VERSION = 1 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ("error_code", Int16), + ( + "groups", + Array(("group", String("utf-8")), ("protocol_type", String("utf-8"))), + ), + ) + + +class ListGroupsResponse_v2(Response): + API_KEY = 16 + API_VERSION = 2 + SCHEMA = ListGroupsResponse_v1.SCHEMA + + +class ListGroupsRequest_v0(Request): + API_KEY = 16 + API_VERSION = 0 + RESPONSE_TYPE = ListGroupsResponse_v0 + SCHEMA = Schema() + + +class ListGroupsRequest_v1(Request): + API_KEY = 16 + API_VERSION = 1 + RESPONSE_TYPE = ListGroupsResponse_v1 + SCHEMA = ListGroupsRequest_v0.SCHEMA + + +class ListGroupsRequest_v2(Request): + API_KEY = 16 + API_VERSION = 1 + RESPONSE_TYPE = ListGroupsResponse_v2 + SCHEMA = ListGroupsRequest_v0.SCHEMA + + +ListGroupsRequest = [ + ListGroupsRequest_v0, + ListGroupsRequest_v1, + ListGroupsRequest_v2, +] +ListGroupsResponse = [ + ListGroupsResponse_v0, + ListGroupsResponse_v1, + ListGroupsResponse_v2, +] + + +class DescribeGroupsResponse_v0(Response): + API_KEY = 15 + API_VERSION = 0 + SCHEMA = Schema( + ( + "groups", + Array( + ("error_code", Int16), + ("group", String("utf-8")), + ("state", String("utf-8")), + ("protocol_type", String("utf-8")), + ("protocol", String("utf-8")), + ( + "members", + Array( + ("member_id", String("utf-8")), + ("client_id", String("utf-8")), + ("client_host", String("utf-8")), + ("member_metadata", Bytes), + ("member_assignment", Bytes), + ), + ), + ), + ) + ) + + +class DescribeGroupsResponse_v1(Response): + API_KEY = 15 + API_VERSION = 1 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "groups", + Array( + ("error_code", Int16), + ("group", String("utf-8")), + ("state", String("utf-8")), + ("protocol_type", String("utf-8")), + ("protocol", String("utf-8")), + ( + "members", + Array( + ("member_id", String("utf-8")), + ("client_id", String("utf-8")), + ("client_host", String("utf-8")), + ("member_metadata", Bytes), + ("member_assignment", Bytes), + ), + ), + ), + ), + ) + + +class DescribeGroupsResponse_v2(Response): + API_KEY = 15 + API_VERSION = 2 + SCHEMA = DescribeGroupsResponse_v1.SCHEMA + + +class DescribeGroupsResponse_v3(Response): + API_KEY = 15 + API_VERSION = 3 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "groups", + Array( + ("error_code", Int16), + ("group", String("utf-8")), + ("state", String("utf-8")), + ("protocol_type", String("utf-8")), + ("protocol", String("utf-8")), + ( + "members", + Array( + ("member_id", String("utf-8")), + ("client_id", String("utf-8")), + ("client_host", String("utf-8")), + ("member_metadata", Bytes), + ("member_assignment", Bytes), + ), + ), + ), + ("authorized_operations", Int32), + ), + ) + + +class DescribeGroupsRequest_v0(Request): + API_KEY = 15 + API_VERSION = 0 + RESPONSE_TYPE = DescribeGroupsResponse_v0 + SCHEMA = Schema(("groups", Array(String("utf-8")))) + + +class DescribeGroupsRequest_v1(Request): + API_KEY = 15 + API_VERSION = 1 + RESPONSE_TYPE = DescribeGroupsResponse_v1 + SCHEMA = DescribeGroupsRequest_v0.SCHEMA + + +class DescribeGroupsRequest_v2(Request): + API_KEY = 15 + API_VERSION = 2 + RESPONSE_TYPE = DescribeGroupsResponse_v2 + SCHEMA = DescribeGroupsRequest_v0.SCHEMA + + +class DescribeGroupsRequest_v3(Request): + API_KEY = 15 + API_VERSION = 3 + RESPONSE_TYPE = DescribeGroupsResponse_v2 + SCHEMA = Schema( + ("groups", Array(String("utf-8"))), ("include_authorized_operations", Boolean) + ) + + +DescribeGroupsRequest = [ + DescribeGroupsRequest_v0, + DescribeGroupsRequest_v1, + DescribeGroupsRequest_v2, + DescribeGroupsRequest_v3, +] +DescribeGroupsResponse = [ + DescribeGroupsResponse_v0, + DescribeGroupsResponse_v1, + DescribeGroupsResponse_v2, + DescribeGroupsResponse_v3, +] + + +class SaslHandShakeResponse_v0(Response): + API_KEY = 17 + API_VERSION = 0 + SCHEMA = Schema( + ("error_code", Int16), ("enabled_mechanisms", Array(String("utf-8"))) + ) + + +class SaslHandShakeResponse_v1(Response): + API_KEY = 17 + API_VERSION = 1 + SCHEMA = SaslHandShakeResponse_v0.SCHEMA + + +class SaslHandShakeRequest_v0(Request): + API_KEY = 17 + API_VERSION = 0 + RESPONSE_TYPE = SaslHandShakeResponse_v0 + SCHEMA = Schema(("mechanism", String("utf-8"))) + + +class SaslHandShakeRequest_v1(Request): + API_KEY = 17 + API_VERSION = 1 + RESPONSE_TYPE = SaslHandShakeResponse_v1 + SCHEMA = SaslHandShakeRequest_v0.SCHEMA + + +SaslHandShakeRequest = [SaslHandShakeRequest_v0, SaslHandShakeRequest_v1] +SaslHandShakeResponse = [SaslHandShakeResponse_v0, SaslHandShakeResponse_v1] + + +class DescribeAclsResponse_v0(Response): + API_KEY = 29 + API_VERSION = 0 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ("error_code", Int16), + ("error_message", String("utf-8")), + ( + "resources", + Array( + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ( + "acls", + Array( + ("principal", String("utf-8")), + ("host", String("utf-8")), + ("operation", Int8), + ("permission_type", Int8), + ), + ), + ), + ), + ) + + +class DescribeAclsResponse_v1(Response): + API_KEY = 29 + API_VERSION = 1 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ("error_code", Int16), + ("error_message", String("utf-8")), + ( + "resources", + Array( + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ("resource_pattern_type", Int8), + ( + "acls", + Array( + ("principal", String("utf-8")), + ("host", String("utf-8")), + ("operation", Int8), + ("permission_type", Int8), + ), + ), + ), + ), + ) + + +class DescribeAclsResponse_v2(Response): + API_KEY = 29 + API_VERSION = 2 + SCHEMA = DescribeAclsResponse_v1.SCHEMA + + +class DescribeAclsRequest_v0(Request): + API_KEY = 29 + API_VERSION = 0 + RESPONSE_TYPE = DescribeAclsResponse_v0 + SCHEMA = Schema( + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ("principal", String("utf-8")), + ("host", String("utf-8")), + ("operation", Int8), + ("permission_type", Int8), + ) + + +class DescribeAclsRequest_v1(Request): + API_KEY = 29 + API_VERSION = 1 + RESPONSE_TYPE = DescribeAclsResponse_v1 + SCHEMA = Schema( + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ("resource_pattern_type_filter", Int8), + ("principal", String("utf-8")), + ("host", String("utf-8")), + ("operation", Int8), + ("permission_type", Int8), + ) + + +class DescribeAclsRequest_v2(Request): + """ + Enable flexible version + """ + + API_KEY = 29 + API_VERSION = 2 + RESPONSE_TYPE = DescribeAclsResponse_v2 + SCHEMA = DescribeAclsRequest_v1.SCHEMA + + +DescribeAclsRequest = [DescribeAclsRequest_v0, DescribeAclsRequest_v1] +DescribeAclsResponse = [DescribeAclsResponse_v0, DescribeAclsResponse_v1] + + +class CreateAclsResponse_v0(Response): + API_KEY = 30 + API_VERSION = 0 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "creation_responses", + Array(("error_code", Int16), ("error_message", String("utf-8"))), + ), + ) + + +class CreateAclsResponse_v1(Response): + API_KEY = 30 + API_VERSION = 1 + SCHEMA = CreateAclsResponse_v0.SCHEMA + + +class CreateAclsRequest_v0(Request): + API_KEY = 30 + API_VERSION = 0 + RESPONSE_TYPE = CreateAclsResponse_v0 + SCHEMA = Schema( + ( + "creations", + Array( + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ("principal", String("utf-8")), + ("host", String("utf-8")), + ("operation", Int8), + ("permission_type", Int8), + ), + ) + ) + + +class CreateAclsRequest_v1(Request): + API_KEY = 30 + API_VERSION = 1 + RESPONSE_TYPE = CreateAclsResponse_v1 + SCHEMA = Schema( + ( + "creations", + Array( + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ("resource_pattern_type", Int8), + ("principal", String("utf-8")), + ("host", String("utf-8")), + ("operation", Int8), + ("permission_type", Int8), + ), + ) + ) + + +CreateAclsRequest = [CreateAclsRequest_v0, CreateAclsRequest_v1] +CreateAclsResponse = [CreateAclsResponse_v0, CreateAclsResponse_v1] + + +class DeleteAclsResponse_v0(Response): + API_KEY = 31 + API_VERSION = 0 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "filter_responses", + Array( + ("error_code", Int16), + ("error_message", String("utf-8")), + ( + "matching_acls", + Array( + ("error_code", Int16), + ("error_message", String("utf-8")), + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ("principal", String("utf-8")), + ("host", String("utf-8")), + ("operation", Int8), + ("permission_type", Int8), + ), + ), + ), + ), + ) + + +class DeleteAclsResponse_v1(Response): + API_KEY = 31 + API_VERSION = 1 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "filter_responses", + Array( + ("error_code", Int16), + ("error_message", String("utf-8")), + ( + "matching_acls", + Array( + ("error_code", Int16), + ("error_message", String("utf-8")), + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ("resource_pattern_type", Int8), + ("principal", String("utf-8")), + ("host", String("utf-8")), + ("operation", Int8), + ("permission_type", Int8), + ), + ), + ), + ), + ) + + +class DeleteAclsRequest_v0(Request): + API_KEY = 31 + API_VERSION = 0 + RESPONSE_TYPE = DeleteAclsResponse_v0 + SCHEMA = Schema( + ( + "filters", + Array( + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ("principal", String("utf-8")), + ("host", String("utf-8")), + ("operation", Int8), + ("permission_type", Int8), + ), + ) + ) + + +class DeleteAclsRequest_v1(Request): + API_KEY = 31 + API_VERSION = 1 + RESPONSE_TYPE = DeleteAclsResponse_v1 + SCHEMA = Schema( + ( + "filters", + Array( + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ("resource_pattern_type_filter", Int8), + ("principal", String("utf-8")), + ("host", String("utf-8")), + ("operation", Int8), + ("permission_type", Int8), + ), + ) + ) + + +DeleteAclsRequest = [DeleteAclsRequest_v0, DeleteAclsRequest_v1] +DeleteAclsResponse = [DeleteAclsResponse_v0, DeleteAclsResponse_v1] + + +class AlterConfigsResponse_v0(Response): + API_KEY = 33 + API_VERSION = 0 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "resources", + Array( + ("error_code", Int16), + ("error_message", String("utf-8")), + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ), + ), + ) + + +class AlterConfigsResponse_v1(Response): + API_KEY = 33 + API_VERSION = 1 + SCHEMA = AlterConfigsResponse_v0.SCHEMA + + +class AlterConfigsRequest_v0(Request): + API_KEY = 33 + API_VERSION = 0 + RESPONSE_TYPE = AlterConfigsResponse_v0 + SCHEMA = Schema( + ( + "resources", + Array( + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ( + "config_entries", + Array( + ("config_name", String("utf-8")), + ("config_value", String("utf-8")), + ), + ), + ), + ), + ("validate_only", Boolean), + ) + + +class AlterConfigsRequest_v1(Request): + API_KEY = 33 + API_VERSION = 1 + RESPONSE_TYPE = AlterConfigsResponse_v1 + SCHEMA = AlterConfigsRequest_v0.SCHEMA + + +AlterConfigsRequest = [AlterConfigsRequest_v0, AlterConfigsRequest_v1] +AlterConfigsResponse = [AlterConfigsResponse_v0, AlterConfigsRequest_v1] + + +class DescribeConfigsResponse_v0(Response): + API_KEY = 32 + API_VERSION = 0 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "resources", + Array( + ("error_code", Int16), + ("error_message", String("utf-8")), + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ( + "config_entries", + Array( + ("config_names", String("utf-8")), + ("config_value", String("utf-8")), + ("read_only", Boolean), + ("is_default", Boolean), + ("is_sensitive", Boolean), + ), + ), + ), + ), + ) + + +class DescribeConfigsResponse_v1(Response): + API_KEY = 32 + API_VERSION = 1 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "resources", + Array( + ("error_code", Int16), + ("error_message", String("utf-8")), + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ( + "config_entries", + Array( + ("config_names", String("utf-8")), + ("config_value", String("utf-8")), + ("read_only", Boolean), + ("is_default", Boolean), + ("is_sensitive", Boolean), + ( + "config_synonyms", + Array( + ("config_name", String("utf-8")), + ("config_value", String("utf-8")), + ("config_source", Int8), + ), + ), + ), + ), + ), + ), + ) + + +class DescribeConfigsResponse_v2(Response): + API_KEY = 32 + API_VERSION = 2 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "resources", + Array( + ("error_code", Int16), + ("error_message", String("utf-8")), + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ( + "config_entries", + Array( + ("config_names", String("utf-8")), + ("config_value", String("utf-8")), + ("read_only", Boolean), + ("config_source", Int8), + ("is_sensitive", Boolean), + ( + "config_synonyms", + Array( + ("config_name", String("utf-8")), + ("config_value", String("utf-8")), + ("config_source", Int8), + ), + ), + ), + ), + ), + ), + ) + + +class DescribeConfigsRequest_v0(Request): + API_KEY = 32 + API_VERSION = 0 + RESPONSE_TYPE = DescribeConfigsResponse_v0 + SCHEMA = Schema( + ( + "resources", + Array( + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ("config_names", Array(String("utf-8"))), + ), + ) + ) + + +class DescribeConfigsRequest_v1(Request): + API_KEY = 32 + API_VERSION = 1 + RESPONSE_TYPE = DescribeConfigsResponse_v1 + SCHEMA = Schema( + ( + "resources", + Array( + ("resource_type", Int8), + ("resource_name", String("utf-8")), + ("config_names", Array(String("utf-8"))), + ), + ), + ("include_synonyms", Boolean), + ) + + +class DescribeConfigsRequest_v2(Request): + API_KEY = 32 + API_VERSION = 2 + RESPONSE_TYPE = DescribeConfigsResponse_v2 + SCHEMA = DescribeConfigsRequest_v1.SCHEMA + + +DescribeConfigsRequest = [ + DescribeConfigsRequest_v0, + DescribeConfigsRequest_v1, + DescribeConfigsRequest_v2, +] +DescribeConfigsResponse = [ + DescribeConfigsResponse_v0, + DescribeConfigsResponse_v1, + DescribeConfigsResponse_v2, +] + + +class SaslAuthenticateResponse_v0(Response): + API_KEY = 36 + API_VERSION = 0 + SCHEMA = Schema( + ("error_code", Int16), + ("error_message", String("utf-8")), + ("sasl_auth_bytes", Bytes), + ) + + +class SaslAuthenticateResponse_v1(Response): + API_KEY = 36 + API_VERSION = 1 + SCHEMA = Schema( + ("error_code", Int16), + ("error_message", String("utf-8")), + ("sasl_auth_bytes", Bytes), + ("session_lifetime_ms", Int64), + ) + + +class SaslAuthenticateRequest_v0(Request): + API_KEY = 36 + API_VERSION = 0 + RESPONSE_TYPE = SaslAuthenticateResponse_v0 + SCHEMA = Schema(("sasl_auth_bytes", Bytes)) + + +class SaslAuthenticateRequest_v1(Request): + API_KEY = 36 + API_VERSION = 1 + RESPONSE_TYPE = SaslAuthenticateResponse_v1 + SCHEMA = SaslAuthenticateRequest_v0.SCHEMA + + +SaslAuthenticateRequest = [ + SaslAuthenticateRequest_v0, + SaslAuthenticateRequest_v1, +] +SaslAuthenticateResponse = [ + SaslAuthenticateResponse_v0, + SaslAuthenticateResponse_v1, +] + + +class CreatePartitionsResponse_v0(Response): + API_KEY = 37 + API_VERSION = 0 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "topic_errors", + Array( + ("topic", String("utf-8")), + ("error_code", Int16), + ("error_message", String("utf-8")), + ), + ), + ) + + +class CreatePartitionsResponse_v1(Response): + API_KEY = 37 + API_VERSION = 1 + SCHEMA = CreatePartitionsResponse_v0.SCHEMA + + +class CreatePartitionsRequest_v0(Request): + API_KEY = 37 + API_VERSION = 0 + RESPONSE_TYPE = CreatePartitionsResponse_v0 + SCHEMA = Schema( + ( + "topic_partitions", + Array( + ("topic", String("utf-8")), + ( + "new_partitions", + Schema(("count", Int32), ("assignment", Array(Array(Int32)))), + ), + ), + ), + ("timeout", Int32), + ("validate_only", Boolean), + ) + + +class CreatePartitionsRequest_v1(Request): + API_KEY = 37 + API_VERSION = 1 + SCHEMA = CreatePartitionsRequest_v0.SCHEMA + RESPONSE_TYPE = CreatePartitionsResponse_v1 + + +CreatePartitionsRequest = [ + CreatePartitionsRequest_v0, + CreatePartitionsRequest_v1, +] +CreatePartitionsResponse = [ + CreatePartitionsResponse_v0, + CreatePartitionsResponse_v1, +] + + +class DeleteGroupsResponse_v0(Response): + API_KEY = 42 + API_VERSION = 0 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ("results", Array(("group_id", String("utf-8")), ("error_code", Int16))), + ) + + +class DeleteGroupsResponse_v1(Response): + API_KEY = 42 + API_VERSION = 1 + SCHEMA = DeleteGroupsResponse_v0.SCHEMA + + +class DeleteGroupsRequest_v0(Request): + API_KEY = 42 + API_VERSION = 0 + RESPONSE_TYPE = DeleteGroupsResponse_v0 + SCHEMA = Schema(("groups_names", Array(String("utf-8")))) + + +class DeleteGroupsRequest_v1(Request): + API_KEY = 42 + API_VERSION = 1 + RESPONSE_TYPE = DeleteGroupsResponse_v1 + SCHEMA = DeleteGroupsRequest_v0.SCHEMA + + +DeleteGroupsRequest = [DeleteGroupsRequest_v0, DeleteGroupsRequest_v1] + +DeleteGroupsResponse = [DeleteGroupsResponse_v0, DeleteGroupsResponse_v1] + + +class DescribeClientQuotasResponse_v0(Request): + API_KEY = 48 + API_VERSION = 0 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ("error_code", Int16), + ("error_message", String("utf-8")), + ( + "entries", + Array( + ( + "entity", + Array( + ("entity_type", String("utf-8")), + ("entity_name", String("utf-8")), + ), + ), + ("values", Array(("name", String("utf-8")), ("value", Float64))), + ), + ), + ) + + +class DescribeClientQuotasRequest_v0(Request): + API_KEY = 48 + API_VERSION = 0 + RESPONSE_TYPE = DescribeClientQuotasResponse_v0 + SCHEMA = Schema( + ( + "components", + Array( + ("entity_type", String("utf-8")), + ("match_type", Int8), + ("match", String("utf-8")), + ), + ), + ("strict", Boolean), + ) + + +DescribeClientQuotasRequest = [ + DescribeClientQuotasRequest_v0, +] + +DescribeClientQuotasResponse = [ + DescribeClientQuotasResponse_v0, +] + + +class AlterPartitionReassignmentsResponse_v0(Response): + API_KEY = 45 + API_VERSION = 0 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ("error_code", Int16), + ("error_message", CompactString("utf-8")), + ( + "responses", + CompactArray( + ("name", CompactString("utf-8")), + ( + "partitions", + CompactArray( + ("partition_index", Int32), + ("error_code", Int16), + ("error_message", CompactString("utf-8")), + ("tags", TaggedFields), + ), + ), + ("tags", TaggedFields), + ), + ), + ("tags", TaggedFields), + ) + + +class AlterPartitionReassignmentsRequest_v0(Request): + FLEXIBLE_VERSION = True + API_KEY = 45 + API_VERSION = 0 + RESPONSE_TYPE = AlterPartitionReassignmentsResponse_v0 + SCHEMA = Schema( + ("timeout_ms", Int32), + ( + "topics", + CompactArray( + ("name", CompactString("utf-8")), + ( + "partitions", + CompactArray( + ("partition_index", Int32), + ("replicas", CompactArray(Int32)), + ("tags", TaggedFields), + ), + ), + ("tags", TaggedFields), + ), + ), + ("tags", TaggedFields), + ) + + +AlterPartitionReassignmentsRequest = [AlterPartitionReassignmentsRequest_v0] + +AlterPartitionReassignmentsResponse = [AlterPartitionReassignmentsResponse_v0] + + +class ListPartitionReassignmentsResponse_v0(Response): + API_KEY = 46 + API_VERSION = 0 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ("error_code", Int16), + ("error_message", CompactString("utf-8")), + ( + "topics", + CompactArray( + ("name", CompactString("utf-8")), + ( + "partitions", + CompactArray( + ("partition_index", Int32), + ("replicas", CompactArray(Int32)), + ("adding_replicas", CompactArray(Int32)), + ("removing_replicas", CompactArray(Int32)), + ("tags", TaggedFields), + ), + ), + ("tags", TaggedFields), + ), + ), + ("tags", TaggedFields), + ) + + +class ListPartitionReassignmentsRequest_v0(Request): + FLEXIBLE_VERSION = True + API_KEY = 46 + API_VERSION = 0 + RESPONSE_TYPE = ListPartitionReassignmentsResponse_v0 + SCHEMA = Schema( + ("timeout_ms", Int32), + ( + "topics", + CompactArray( + ("name", CompactString("utf-8")), + ("partition_index", CompactArray(Int32)), + ("tags", TaggedFields), + ), + ), + ("tags", TaggedFields), + ) + + +ListPartitionReassignmentsRequest = [ListPartitionReassignmentsRequest_v0] + +ListPartitionReassignmentsResponse = [ListPartitionReassignmentsResponse_v0] diff --git a/kafka/protocol/api.py b/aiokafka/protocol/api.py similarity index 74% rename from kafka/protocol/api.py rename to aiokafka/protocol/api.py index f12cb972..9eb2b6fe 100644 --- a/kafka/protocol/api.py +++ b/aiokafka/protocol/api.py @@ -1,20 +1,18 @@ -from __future__ import absolute_import - import abc -from kafka.protocol.struct import Struct -from kafka.protocol.types import Int16, Int32, String, Schema, Array, TaggedFields +from .struct import Struct +from .types import Int16, Int32, String, Schema, Array, TaggedFields class RequestHeader(Struct): SCHEMA = Schema( - ('api_key', Int16), - ('api_version', Int16), - ('correlation_id', Int32), - ('client_id', String('utf-8')) + ("api_key", Int16), + ("api_version", Int16), + ("correlation_id", Int32), + ("client_id", String("utf-8")), ) - def __init__(self, request, correlation_id=0, client_id='kafka-python'): + def __init__(self, request, correlation_id=0, client_id="kafka-python"): super(RequestHeader, self).__init__( request.API_KEY, request.API_VERSION, correlation_id, client_id ) @@ -23,14 +21,14 @@ def __init__(self, request, correlation_id=0, client_id='kafka-python'): class RequestHeaderV2(Struct): # Flexible response / request headers end in field buffer SCHEMA = Schema( - ('api_key', Int16), - ('api_version', Int16), - ('correlation_id', Int32), - ('client_id', String('utf-8')), - ('tags', TaggedFields), + ("api_key", Int16), + ("api_version", Int16), + ("correlation_id", Int32), + ("client_id", String("utf-8")), + ("tags", TaggedFields), ) - def __init__(self, request, correlation_id=0, client_id='kafka-python', tags=None): + def __init__(self, request, correlation_id=0, client_id="kafka-python", tags=None): super(RequestHeaderV2, self).__init__( request.API_KEY, request.API_VERSION, correlation_id, client_id, tags or {} ) @@ -38,14 +36,14 @@ def __init__(self, request, correlation_id=0, client_id='kafka-python', tags=Non class ResponseHeader(Struct): SCHEMA = Schema( - ('correlation_id', Int32), + ("correlation_id", Int32), ) class ResponseHeaderV2(Struct): SCHEMA = Schema( - ('correlation_id', Int32), - ('tags', TaggedFields), + ("correlation_id", Int32), + ("tags", TaggedFields), ) @@ -83,7 +81,9 @@ def to_object(self): def build_request_header(self, correlation_id, client_id): if self.FLEXIBLE_VERSION: - return RequestHeaderV2(self, correlation_id=correlation_id, client_id=client_id) + return RequestHeaderV2( + self, correlation_id=correlation_id, client_id=client_id + ) return RequestHeader(self, correlation_id=correlation_id, client_id=client_id) def parse_response_header(self, read_buffer): @@ -126,10 +126,7 @@ def _to_object(schema, data): obj[name] = _to_object(_type, val) elif isinstance(_type, Array): if isinstance(_type.array_of, (Array, Schema)): - obj[name] = [ - _to_object(_type.array_of, x) - for x in val - ] + obj[name] = [_to_object(_type.array_of, x) for x in val] else: obj[name] = val else: diff --git a/aiokafka/protocol/commit.py b/aiokafka/protocol/commit.py new file mode 100644 index 00000000..81185397 --- /dev/null +++ b/aiokafka/protocol/commit.py @@ -0,0 +1,306 @@ +from .api import Request, Response +from .types import Array, Int8, Int16, Int32, Int64, Schema, String + + +class OffsetCommitResponse_v0(Response): + API_KEY = 8 + API_VERSION = 0 + SCHEMA = Schema( + ( + "topics", + Array( + ("topic", String("utf-8")), + ("partitions", Array(("partition", Int32), ("error_code", Int16))), + ), + ) + ) + + +class OffsetCommitResponse_v1(Response): + API_KEY = 8 + API_VERSION = 1 + SCHEMA = OffsetCommitResponse_v0.SCHEMA + + +class OffsetCommitResponse_v2(Response): + API_KEY = 8 + API_VERSION = 2 + SCHEMA = OffsetCommitResponse_v1.SCHEMA + + +class OffsetCommitResponse_v3(Response): + API_KEY = 8 + API_VERSION = 3 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "topics", + Array( + ("topic", String("utf-8")), + ("partitions", Array(("partition", Int32), ("error_code", Int16))), + ), + ), + ) + + +class OffsetCommitRequest_v0(Request): + API_KEY = 8 + API_VERSION = 0 # Zookeeper-backed storage + RESPONSE_TYPE = OffsetCommitResponse_v0 + SCHEMA = Schema( + ("consumer_group", String("utf-8")), + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("offset", Int64), + ("metadata", String("utf-8")), + ), + ), + ), + ), + ) + + +class OffsetCommitRequest_v1(Request): + API_KEY = 8 + API_VERSION = 1 # Kafka-backed storage + RESPONSE_TYPE = OffsetCommitResponse_v1 + SCHEMA = Schema( + ("consumer_group", String("utf-8")), + ("consumer_group_generation_id", Int32), + ("consumer_id", String("utf-8")), + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("offset", Int64), + ("timestamp", Int64), + ("metadata", String("utf-8")), + ), + ), + ), + ), + ) + + +class OffsetCommitRequest_v2(Request): + API_KEY = 8 + API_VERSION = 2 # added retention_time, dropped timestamp + RESPONSE_TYPE = OffsetCommitResponse_v2 + SCHEMA = Schema( + ("consumer_group", String("utf-8")), + ("consumer_group_generation_id", Int32), + ("consumer_id", String("utf-8")), + ("retention_time", Int64), + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("offset", Int64), + ("metadata", String("utf-8")), + ), + ), + ), + ), + ) + DEFAULT_GENERATION_ID = -1 + DEFAULT_RETENTION_TIME = -1 + + +class OffsetCommitRequest_v3(Request): + API_KEY = 8 + API_VERSION = 3 + RESPONSE_TYPE = OffsetCommitResponse_v3 + SCHEMA = OffsetCommitRequest_v2.SCHEMA + + +OffsetCommitRequest = [ + OffsetCommitRequest_v0, + OffsetCommitRequest_v1, + OffsetCommitRequest_v2, + OffsetCommitRequest_v3, +] +OffsetCommitResponse = [ + OffsetCommitResponse_v0, + OffsetCommitResponse_v1, + OffsetCommitResponse_v2, + OffsetCommitResponse_v3, +] + + +class OffsetFetchResponse_v0(Response): + API_KEY = 9 + API_VERSION = 0 + SCHEMA = Schema( + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("offset", Int64), + ("metadata", String("utf-8")), + ("error_code", Int16), + ), + ), + ), + ) + ) + + +class OffsetFetchResponse_v1(Response): + API_KEY = 9 + API_VERSION = 1 + SCHEMA = OffsetFetchResponse_v0.SCHEMA + + +class OffsetFetchResponse_v2(Response): + # Added in KIP-88 + API_KEY = 9 + API_VERSION = 2 + SCHEMA = Schema( + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("offset", Int64), + ("metadata", String("utf-8")), + ("error_code", Int16), + ), + ), + ), + ), + ("error_code", Int16), + ) + + +class OffsetFetchResponse_v3(Response): + API_KEY = 9 + API_VERSION = 3 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("offset", Int64), + ("metadata", String("utf-8")), + ("error_code", Int16), + ), + ), + ), + ), + ("error_code", Int16), + ) + + +class OffsetFetchRequest_v0(Request): + API_KEY = 9 + API_VERSION = 0 # zookeeper-backed storage + RESPONSE_TYPE = OffsetFetchResponse_v0 + SCHEMA = Schema( + ("consumer_group", String("utf-8")), + ("topics", Array(("topic", String("utf-8")), ("partitions", Array(Int32)))), + ) + + +class OffsetFetchRequest_v1(Request): + API_KEY = 9 + API_VERSION = 1 # kafka-backed storage + RESPONSE_TYPE = OffsetFetchResponse_v1 + SCHEMA = OffsetFetchRequest_v0.SCHEMA + + +class OffsetFetchRequest_v2(Request): + # KIP-88: Allows passing null topics to return offsets for all partitions + # that the consumer group has a stored offset for, even if no consumer in + # the group is currently consuming that partition. + API_KEY = 9 + API_VERSION = 2 + RESPONSE_TYPE = OffsetFetchResponse_v2 + SCHEMA = OffsetFetchRequest_v1.SCHEMA + + +class OffsetFetchRequest_v3(Request): + API_KEY = 9 + API_VERSION = 3 + RESPONSE_TYPE = OffsetFetchResponse_v3 + SCHEMA = OffsetFetchRequest_v2.SCHEMA + + +OffsetFetchRequest = [ + OffsetFetchRequest_v0, + OffsetFetchRequest_v1, + OffsetFetchRequest_v2, + OffsetFetchRequest_v3, +] +OffsetFetchResponse = [ + OffsetFetchResponse_v0, + OffsetFetchResponse_v1, + OffsetFetchResponse_v2, + OffsetFetchResponse_v3, +] + + +class GroupCoordinatorResponse_v0(Response): + API_KEY = 10 + API_VERSION = 0 + SCHEMA = Schema( + ("error_code", Int16), + ("coordinator_id", Int32), + ("host", String("utf-8")), + ("port", Int32), + ) + + +class GroupCoordinatorResponse_v1(Response): + API_KEY = 10 + API_VERSION = 1 + SCHEMA = Schema( + ("error_code", Int16), + ("error_message", String("utf-8")), + ("coordinator_id", Int32), + ("host", String("utf-8")), + ("port", Int32), + ) + + +class GroupCoordinatorRequest_v0(Request): + API_KEY = 10 + API_VERSION = 0 + RESPONSE_TYPE = GroupCoordinatorResponse_v0 + SCHEMA = Schema(("consumer_group", String("utf-8"))) + + +class GroupCoordinatorRequest_v1(Request): + API_KEY = 10 + API_VERSION = 1 + RESPONSE_TYPE = GroupCoordinatorResponse_v1 + SCHEMA = Schema(("coordinator_key", String("utf-8")), ("coordinator_type", Int8)) + + +GroupCoordinatorRequest = [GroupCoordinatorRequest_v0, GroupCoordinatorRequest_v1] +GroupCoordinatorResponse = [GroupCoordinatorResponse_v0, GroupCoordinatorResponse_v1] diff --git a/aiokafka/protocol/coordination.py b/aiokafka/protocol/coordination.py index 1690b9ae..9bf086ac 100644 --- a/aiokafka/protocol/coordination.py +++ b/aiokafka/protocol/coordination.py @@ -1,15 +1,15 @@ -from kafka.protocol.api import Request, Response -from kafka.protocol.types import Int8, Int16, Int32, Schema, String +from .api import Request, Response +from .types import Int8, Int16, Int32, Schema, String class FindCoordinatorResponse_v0(Response): API_KEY = 10 API_VERSION = 0 SCHEMA = Schema( - ('error_code', Int16), - ('coordinator_id', Int32), - ('host', String('utf-8')), - ('port', Int32) + ("error_code", Int16), + ("coordinator_id", Int32), + ("host", String("utf-8")), + ("port", Int32), ) @@ -17,12 +17,12 @@ class FindCoordinatorResponse_v1(Response): API_KEY = 10 API_VERSION = 1 SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('error_code', Int16), - ('error_message', String('utf-8')), - ('coordinator_id', Int32), - ('host', String('utf-8')), - ('port', Int32) + ("throttle_time_ms", Int32), + ("error_code", Int16), + ("error_message", String("utf-8")), + ("coordinator_id", Int32), + ("host", String("utf-8")), + ("port", Int32), ) @@ -30,22 +30,15 @@ class FindCoordinatorRequest_v0(Request): API_KEY = 10 API_VERSION = 0 RESPONSE_TYPE = FindCoordinatorResponse_v0 - SCHEMA = Schema( - ('consumer_group', String('utf-8')) - ) + SCHEMA = Schema(("consumer_group", String("utf-8"))) class FindCoordinatorRequest_v1(Request): API_KEY = 10 API_VERSION = 1 RESPONSE_TYPE = FindCoordinatorResponse_v1 - SCHEMA = Schema( - ('coordinator_key', String('utf-8')), - ('coordinator_type', Int8) - ) + SCHEMA = Schema(("coordinator_key", String("utf-8")), ("coordinator_type", Int8)) -FindCoordinatorRequest = [ - FindCoordinatorRequest_v0, FindCoordinatorRequest_v1] -FindCoordinatorResponse = [ - FindCoordinatorResponse_v0, FindCoordinatorResponse_v1] +FindCoordinatorRequest = [FindCoordinatorRequest_v0, FindCoordinatorRequest_v1] +FindCoordinatorResponse = [FindCoordinatorResponse_v0, FindCoordinatorResponse_v1] diff --git a/aiokafka/protocol/fetch.py b/aiokafka/protocol/fetch.py new file mode 100644 index 00000000..be5518e2 --- /dev/null +++ b/aiokafka/protocol/fetch.py @@ -0,0 +1,516 @@ +from .api import Request, Response +from .types import Array, Int8, Int16, Int32, Int64, Schema, String, Bytes + + +class FetchResponse_v0(Response): + API_KEY = 1 + API_VERSION = 0 + SCHEMA = Schema( + ( + "topics", + Array( + ("topics", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("error_code", Int16), + ("highwater_offset", Int64), + ("message_set", Bytes), + ), + ), + ), + ) + ) + + +class FetchResponse_v1(Response): + API_KEY = 1 + API_VERSION = 1 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "topics", + Array( + ("topics", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("error_code", Int16), + ("highwater_offset", Int64), + ("message_set", Bytes), + ), + ), + ), + ), + ) + + +class FetchResponse_v2(Response): + API_KEY = 1 + API_VERSION = 2 + SCHEMA = FetchResponse_v1.SCHEMA # message format changed internally + + +class FetchResponse_v3(Response): + API_KEY = 1 + API_VERSION = 3 + SCHEMA = FetchResponse_v2.SCHEMA + + +class FetchResponse_v4(Response): + API_KEY = 1 + API_VERSION = 4 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "topics", + Array( + ("topics", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("error_code", Int16), + ("highwater_offset", Int64), + ("last_stable_offset", Int64), + ( + "aborted_transactions", + Array(("producer_id", Int64), ("first_offset", Int64)), + ), + ("message_set", Bytes), + ), + ), + ), + ), + ) + + +class FetchResponse_v5(Response): + API_KEY = 1 + API_VERSION = 5 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "topics", + Array( + ("topics", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("error_code", Int16), + ("highwater_offset", Int64), + ("last_stable_offset", Int64), + ("log_start_offset", Int64), + ( + "aborted_transactions", + Array(("producer_id", Int64), ("first_offset", Int64)), + ), + ("message_set", Bytes), + ), + ), + ), + ), + ) + + +class FetchResponse_v6(Response): + """ + Same as FetchResponse_v5. The version number is bumped up to indicate that the + client supports KafkaStorageException. The KafkaStorageException will be translated + to NotLeaderForPartitionException in the response if version <= 5 + """ + + API_KEY = 1 + API_VERSION = 6 + SCHEMA = FetchResponse_v5.SCHEMA + + +class FetchResponse_v7(Response): + """ + Add error_code and session_id to response + """ + + API_KEY = 1 + API_VERSION = 7 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ("error_code", Int16), + ("session_id", Int32), + ( + "topics", + Array( + ("topics", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("error_code", Int16), + ("highwater_offset", Int64), + ("last_stable_offset", Int64), + ("log_start_offset", Int64), + ( + "aborted_transactions", + Array(("producer_id", Int64), ("first_offset", Int64)), + ), + ("message_set", Bytes), + ), + ), + ), + ), + ) + + +class FetchResponse_v8(Response): + API_KEY = 1 + API_VERSION = 8 + SCHEMA = FetchResponse_v7.SCHEMA + + +class FetchResponse_v9(Response): + API_KEY = 1 + API_VERSION = 9 + SCHEMA = FetchResponse_v7.SCHEMA + + +class FetchResponse_v10(Response): + API_KEY = 1 + API_VERSION = 10 + SCHEMA = FetchResponse_v7.SCHEMA + + +class FetchResponse_v11(Response): + API_KEY = 1 + API_VERSION = 11 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ("error_code", Int16), + ("session_id", Int32), + ( + "topics", + Array( + ("topics", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("error_code", Int16), + ("highwater_offset", Int64), + ("last_stable_offset", Int64), + ("log_start_offset", Int64), + ( + "aborted_transactions", + Array(("producer_id", Int64), ("first_offset", Int64)), + ), + ("preferred_read_replica", Int32), + ("message_set", Bytes), + ), + ), + ), + ), + ) + + +class FetchRequest_v0(Request): + API_KEY = 1 + API_VERSION = 0 + RESPONSE_TYPE = FetchResponse_v0 + SCHEMA = Schema( + ("replica_id", Int32), + ("max_wait_time", Int32), + ("min_bytes", Int32), + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), ("offset", Int64), ("max_bytes", Int32) + ), + ), + ), + ), + ) + + +class FetchRequest_v1(Request): + API_KEY = 1 + API_VERSION = 1 + RESPONSE_TYPE = FetchResponse_v1 + SCHEMA = FetchRequest_v0.SCHEMA + + +class FetchRequest_v2(Request): + API_KEY = 1 + API_VERSION = 2 + RESPONSE_TYPE = FetchResponse_v2 + SCHEMA = FetchRequest_v1.SCHEMA + + +class FetchRequest_v3(Request): + API_KEY = 1 + API_VERSION = 3 + RESPONSE_TYPE = FetchResponse_v3 + SCHEMA = Schema( + ("replica_id", Int32), + ("max_wait_time", Int32), + ("min_bytes", Int32), + ("max_bytes", Int32), # This new field is only difference from FR_v2 + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), ("offset", Int64), ("max_bytes", Int32) + ), + ), + ), + ), + ) + + +class FetchRequest_v4(Request): + # Adds isolation_level field + API_KEY = 1 + API_VERSION = 4 + RESPONSE_TYPE = FetchResponse_v4 + SCHEMA = Schema( + ("replica_id", Int32), + ("max_wait_time", Int32), + ("min_bytes", Int32), + ("max_bytes", Int32), + ("isolation_level", Int8), + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), ("offset", Int64), ("max_bytes", Int32) + ), + ), + ), + ), + ) + + +class FetchRequest_v5(Request): + # This may only be used in broker-broker api calls + API_KEY = 1 + API_VERSION = 5 + RESPONSE_TYPE = FetchResponse_v5 + SCHEMA = Schema( + ("replica_id", Int32), + ("max_wait_time", Int32), + ("min_bytes", Int32), + ("max_bytes", Int32), + ("isolation_level", Int8), + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("fetch_offset", Int64), + ("log_start_offset", Int64), + ("max_bytes", Int32), + ), + ), + ), + ), + ) + + +class FetchRequest_v6(Request): + """ + The body of FETCH_REQUEST_V6 is the same as FETCH_REQUEST_V5. The version number is + bumped up to indicate that the client supports KafkaStorageException. The + KafkaStorageException will be translated to NotLeaderForPartitionException in the + response if version <= 5 + """ + + API_KEY = 1 + API_VERSION = 6 + RESPONSE_TYPE = FetchResponse_v6 + SCHEMA = FetchRequest_v5.SCHEMA + + +class FetchRequest_v7(Request): + """ + Add incremental fetch requests + """ + + API_KEY = 1 + API_VERSION = 7 + RESPONSE_TYPE = FetchResponse_v7 + SCHEMA = Schema( + ("replica_id", Int32), + ("max_wait_time", Int32), + ("min_bytes", Int32), + ("max_bytes", Int32), + ("isolation_level", Int8), + ("session_id", Int32), + ("session_epoch", Int32), + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("fetch_offset", Int64), + ("log_start_offset", Int64), + ("max_bytes", Int32), + ), + ), + ), + ), + ( + "forgotten_topics_data", + Array(("topic", String), ("partitions", Array(Int32))), + ), + ) + + +class FetchRequest_v8(Request): + """ + bump used to indicate that on quota violation brokers send out responses before + throttling. + """ + + API_KEY = 1 + API_VERSION = 8 + RESPONSE_TYPE = FetchResponse_v8 + SCHEMA = FetchRequest_v7.SCHEMA + + +class FetchRequest_v9(Request): + """ + adds the current leader epoch (see KIP-320) + """ + + API_KEY = 1 + API_VERSION = 9 + RESPONSE_TYPE = FetchResponse_v9 + SCHEMA = Schema( + ("replica_id", Int32), + ("max_wait_time", Int32), + ("min_bytes", Int32), + ("max_bytes", Int32), + ("isolation_level", Int8), + ("session_id", Int32), + ("session_epoch", Int32), + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("current_leader_epoch", Int32), + ("fetch_offset", Int64), + ("log_start_offset", Int64), + ("max_bytes", Int32), + ), + ), + ), + ), + ( + "forgotten_topics_data", + Array( + ("topic", String), + ("partitions", Array(Int32)), + ), + ), + ) + + +class FetchRequest_v10(Request): + """ + bumped up to indicate ZStandard capability. (see KIP-110) + """ + + API_KEY = 1 + API_VERSION = 10 + RESPONSE_TYPE = FetchResponse_v10 + SCHEMA = FetchRequest_v9.SCHEMA + + +class FetchRequest_v11(Request): + """ + added rack ID to support read from followers (KIP-392) + """ + + API_KEY = 1 + API_VERSION = 11 + RESPONSE_TYPE = FetchResponse_v11 + SCHEMA = Schema( + ("replica_id", Int32), + ("max_wait_time", Int32), + ("min_bytes", Int32), + ("max_bytes", Int32), + ("isolation_level", Int8), + ("session_id", Int32), + ("session_epoch", Int32), + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("current_leader_epoch", Int32), + ("fetch_offset", Int64), + ("log_start_offset", Int64), + ("max_bytes", Int32), + ), + ), + ), + ), + ( + "forgotten_topics_data", + Array(("topic", String), ("partitions", Array(Int32))), + ), + ("rack_id", String("utf-8")), + ) + + +FetchRequest = [ + FetchRequest_v0, + FetchRequest_v1, + FetchRequest_v2, + FetchRequest_v3, + FetchRequest_v4, + FetchRequest_v5, + FetchRequest_v6, + FetchRequest_v7, + FetchRequest_v8, + FetchRequest_v9, + FetchRequest_v10, + FetchRequest_v11, +] +FetchResponse = [ + FetchResponse_v0, + FetchResponse_v1, + FetchResponse_v2, + FetchResponse_v3, + FetchResponse_v4, + FetchResponse_v5, + FetchResponse_v6, + FetchResponse_v7, + FetchResponse_v8, + FetchResponse_v9, + FetchResponse_v10, + FetchResponse_v11, +] diff --git a/kafka/protocol/frame.py b/aiokafka/protocol/frame.py similarity index 94% rename from kafka/protocol/frame.py rename to aiokafka/protocol/frame.py index 7b4a32bc..897e091b 100644 --- a/kafka/protocol/frame.py +++ b/aiokafka/protocol/frame.py @@ -24,7 +24,7 @@ def tell(self): return self._idx def __str__(self): - return 'KafkaBytes(%d)' % len(self) + return "KafkaBytes(%d)" % len(self) def __repr__(self): return str(self) diff --git a/aiokafka/protocol/group.py b/aiokafka/protocol/group.py new file mode 100644 index 00000000..a809738a --- /dev/null +++ b/aiokafka/protocol/group.py @@ -0,0 +1,203 @@ +from .api import Request, Response +from .struct import Struct +from .types import Array, Bytes, Int16, Int32, Schema, String + + +class JoinGroupResponse_v0(Response): + API_KEY = 11 + API_VERSION = 0 + SCHEMA = Schema( + ("error_code", Int16), + ("generation_id", Int32), + ("group_protocol", String("utf-8")), + ("leader_id", String("utf-8")), + ("member_id", String("utf-8")), + ("members", Array(("member_id", String("utf-8")), ("member_metadata", Bytes))), + ) + + +class JoinGroupResponse_v1(Response): + API_KEY = 11 + API_VERSION = 1 + SCHEMA = JoinGroupResponse_v0.SCHEMA + + +class JoinGroupResponse_v2(Response): + API_KEY = 11 + API_VERSION = 2 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ("error_code", Int16), + ("generation_id", Int32), + ("group_protocol", String("utf-8")), + ("leader_id", String("utf-8")), + ("member_id", String("utf-8")), + ("members", Array(("member_id", String("utf-8")), ("member_metadata", Bytes))), + ) + + +class JoinGroupRequest_v0(Request): + API_KEY = 11 + API_VERSION = 0 + RESPONSE_TYPE = JoinGroupResponse_v0 + SCHEMA = Schema( + ("group", String("utf-8")), + ("session_timeout", Int32), + ("member_id", String("utf-8")), + ("protocol_type", String("utf-8")), + ( + "group_protocols", + Array(("protocol_name", String("utf-8")), ("protocol_metadata", Bytes)), + ), + ) + UNKNOWN_MEMBER_ID = "" + + +class JoinGroupRequest_v1(Request): + API_KEY = 11 + API_VERSION = 1 + RESPONSE_TYPE = JoinGroupResponse_v1 + SCHEMA = Schema( + ("group", String("utf-8")), + ("session_timeout", Int32), + ("rebalance_timeout", Int32), + ("member_id", String("utf-8")), + ("protocol_type", String("utf-8")), + ( + "group_protocols", + Array(("protocol_name", String("utf-8")), ("protocol_metadata", Bytes)), + ), + ) + UNKNOWN_MEMBER_ID = "" + + +class JoinGroupRequest_v2(Request): + API_KEY = 11 + API_VERSION = 2 + RESPONSE_TYPE = JoinGroupResponse_v2 + SCHEMA = JoinGroupRequest_v1.SCHEMA + UNKNOWN_MEMBER_ID = "" + + +JoinGroupRequest = [JoinGroupRequest_v0, JoinGroupRequest_v1, JoinGroupRequest_v2] +JoinGroupResponse = [JoinGroupResponse_v0, JoinGroupResponse_v1, JoinGroupResponse_v2] + + +class ProtocolMetadata(Struct): + SCHEMA = Schema( + ("version", Int16), + ("subscription", Array(String("utf-8"))), # topics list + ("user_data", Bytes), + ) + + +class SyncGroupResponse_v0(Response): + API_KEY = 14 + API_VERSION = 0 + SCHEMA = Schema(("error_code", Int16), ("member_assignment", Bytes)) + + +class SyncGroupResponse_v1(Response): + API_KEY = 14 + API_VERSION = 1 + SCHEMA = Schema( + ("throttle_time_ms", Int32), ("error_code", Int16), ("member_assignment", Bytes) + ) + + +class SyncGroupRequest_v0(Request): + API_KEY = 14 + API_VERSION = 0 + RESPONSE_TYPE = SyncGroupResponse_v0 + SCHEMA = Schema( + ("group", String("utf-8")), + ("generation_id", Int32), + ("member_id", String("utf-8")), + ( + "group_assignment", + Array(("member_id", String("utf-8")), ("member_metadata", Bytes)), + ), + ) + + +class SyncGroupRequest_v1(Request): + API_KEY = 14 + API_VERSION = 1 + RESPONSE_TYPE = SyncGroupResponse_v1 + SCHEMA = SyncGroupRequest_v0.SCHEMA + + +SyncGroupRequest = [SyncGroupRequest_v0, SyncGroupRequest_v1] +SyncGroupResponse = [SyncGroupResponse_v0, SyncGroupResponse_v1] + + +class MemberAssignment(Struct): + SCHEMA = Schema( + ("version", Int16), + ("assignment", Array(("topic", String("utf-8")), ("partitions", Array(Int32)))), + ("user_data", Bytes), + ) + + +class HeartbeatResponse_v0(Response): + API_KEY = 12 + API_VERSION = 0 + SCHEMA = Schema(("error_code", Int16)) + + +class HeartbeatResponse_v1(Response): + API_KEY = 12 + API_VERSION = 1 + SCHEMA = Schema(("throttle_time_ms", Int32), ("error_code", Int16)) + + +class HeartbeatRequest_v0(Request): + API_KEY = 12 + API_VERSION = 0 + RESPONSE_TYPE = HeartbeatResponse_v0 + SCHEMA = Schema( + ("group", String("utf-8")), + ("generation_id", Int32), + ("member_id", String("utf-8")), + ) + + +class HeartbeatRequest_v1(Request): + API_KEY = 12 + API_VERSION = 1 + RESPONSE_TYPE = HeartbeatResponse_v1 + SCHEMA = HeartbeatRequest_v0.SCHEMA + + +HeartbeatRequest = [HeartbeatRequest_v0, HeartbeatRequest_v1] +HeartbeatResponse = [HeartbeatResponse_v0, HeartbeatResponse_v1] + + +class LeaveGroupResponse_v0(Response): + API_KEY = 13 + API_VERSION = 0 + SCHEMA = Schema(("error_code", Int16)) + + +class LeaveGroupResponse_v1(Response): + API_KEY = 13 + API_VERSION = 1 + SCHEMA = Schema(("throttle_time_ms", Int32), ("error_code", Int16)) + + +class LeaveGroupRequest_v0(Request): + API_KEY = 13 + API_VERSION = 0 + RESPONSE_TYPE = LeaveGroupResponse_v0 + SCHEMA = Schema(("group", String("utf-8")), ("member_id", String("utf-8"))) + + +class LeaveGroupRequest_v1(Request): + API_KEY = 13 + API_VERSION = 1 + RESPONSE_TYPE = LeaveGroupResponse_v1 + SCHEMA = LeaveGroupRequest_v0.SCHEMA + + +LeaveGroupRequest = [LeaveGroupRequest_v0, LeaveGroupRequest_v1] +LeaveGroupResponse = [LeaveGroupResponse_v0, LeaveGroupResponse_v1] diff --git a/kafka/protocol/message.py b/aiokafka/protocol/message.py similarity index 73% rename from kafka/protocol/message.py rename to aiokafka/protocol/message.py index 4c5c031b..d187b9bc 100644 --- a/kafka/protocol/message.py +++ b/aiokafka/protocol/message.py @@ -1,34 +1,40 @@ -from __future__ import absolute_import - import io import time -from kafka.codec import (has_gzip, has_snappy, has_lz4, has_zstd, - gzip_decode, snappy_decode, zstd_decode, - lz4_decode, lz4_decode_old_kafka) -from kafka.protocol.frame import KafkaBytes -from kafka.protocol.struct import Struct -from kafka.protocol.types import ( - Int8, Int32, Int64, Bytes, Schema, AbstractType +from kafka.codec import ( + has_gzip, + has_snappy, + has_lz4, + has_zstd, + gzip_decode, + snappy_decode, + zstd_decode, + lz4_decode, + lz4_decode_old_kafka, ) +from .frame import KafkaBytes +from .struct import Struct +from .types import Int8, Int32, Int64, Bytes, Schema, AbstractType from kafka.util import crc32, WeakMethod class Message(Struct): SCHEMAS = [ Schema( - ('crc', Int32), - ('magic', Int8), - ('attributes', Int8), - ('key', Bytes), - ('value', Bytes)), + ("crc", Int32), + ("magic", Int8), + ("attributes", Int8), + ("key", Bytes), + ("value", Bytes), + ), Schema( - ('crc', Int32), - ('magic', Int8), - ('attributes', Int8), - ('timestamp', Int64), - ('key', Bytes), - ('value', Bytes)), + ("crc", Int32), + ("magic", Int8), + ("attributes", Int8), + ("timestamp", Int64), + ("key", Bytes), + ("value", Bytes), + ), ] SCHEMA = SCHEMAS[1] CODEC_MASK = 0x07 @@ -37,13 +43,14 @@ class Message(Struct): CODEC_LZ4 = 0x03 CODEC_ZSTD = 0x04 TIMESTAMP_TYPE_MASK = 0x08 - HEADER_SIZE = 22 # crc(4), magic(1), attributes(1), timestamp(8), key+value size(4*2) + HEADER_SIZE = ( + 22 # crc(4), magic(1), attributes(1), timestamp(8), key+value size(4*2) + ) - def __init__(self, value, key=None, magic=0, attributes=0, crc=0, - timestamp=None): - assert value is None or isinstance(value, bytes), 'value must be bytes' - assert key is None or isinstance(key, bytes), 'key must be bytes' - assert magic > 0 or timestamp is None, 'timestamp not supported in v0' + def __init__(self, value, key=None, magic=0, attributes=0, crc=0, timestamp=None): + assert value is None or isinstance(value, bytes), "value must be bytes" + assert key is None or isinstance(key, bytes), "key must be bytes" + assert magic > 0 or timestamp is None, "timestamp not supported in v0" # Default timestamp to now for v1 messages if magic > 0 and timestamp is None: @@ -74,11 +81,18 @@ def timestamp_type(self): def _encode_self(self, recalc_crc=True): version = self.magic if version == 1: - fields = (self.crc, self.magic, self.attributes, self.timestamp, self.key, self.value) + fields = ( + self.crc, + self.magic, + self.attributes, + self.timestamp, + self.key, + self.value, + ) elif version == 0: fields = (self.crc, self.magic, self.attributes, self.key, self.value) else: - raise ValueError('Unrecognized message version: %s' % (version,)) + raise ValueError("Unrecognized message version: %s" % (version,)) message = Message.SCHEMAS[version].encode(fields) if not recalc_crc: return message @@ -101,9 +115,14 @@ def decode(cls, data): timestamp = fields[0] else: timestamp = None - msg = cls(fields[-1], key=fields[-2], - magic=magic, attributes=attributes, crc=crc, - timestamp=timestamp) + msg = cls( + fields[-1], + key=fields[-2], + magic=magic, + attributes=attributes, + crc=crc, + timestamp=timestamp, + ) msg._validated_crc = _validated_crc return msg @@ -120,15 +139,20 @@ def is_compressed(self): def decompress(self): codec = self.attributes & self.CODEC_MASK - assert codec in (self.CODEC_GZIP, self.CODEC_SNAPPY, self.CODEC_LZ4, self.CODEC_ZSTD) + assert codec in ( + self.CODEC_GZIP, + self.CODEC_SNAPPY, + self.CODEC_LZ4, + self.CODEC_ZSTD, + ) if codec == self.CODEC_GZIP: - assert has_gzip(), 'Gzip decompression unsupported' + assert has_gzip(), "Gzip decompression unsupported" raw_bytes = gzip_decode(self.value) elif codec == self.CODEC_SNAPPY: - assert has_snappy(), 'Snappy decompression unsupported' + assert has_snappy(), "Snappy decompression unsupported" raw_bytes = snappy_decode(self.value) elif codec == self.CODEC_LZ4: - assert has_lz4(), 'LZ4 decompression unsupported' + assert has_lz4(), "LZ4 decompression unsupported" if self.magic == 0: raw_bytes = lz4_decode_old_kafka(self.value) else: @@ -137,7 +161,7 @@ def decompress(self): assert has_zstd(), "ZSTD decompression unsupported" raw_bytes = zstd_decode(self.value) else: - raise Exception('This should be impossible') + raise Exception("This should be impossible") return MessageSet.decode(raw_bytes, bytes_to_read=len(raw_bytes)) @@ -147,14 +171,11 @@ def __hash__(self): class PartialMessage(bytes): def __repr__(self): - return 'PartialMessage(%s)' % (self,) + return "PartialMessage(%s)" % (self,) class MessageSet(AbstractType): - ITEM = Schema( - ('offset', Int64), - ('message', Bytes) - ) + ITEM = Schema(("offset", Int64), ("message", Bytes)) HEADER_SIZE = 12 # offset + message_size @classmethod @@ -172,7 +193,7 @@ def encode(cls, items, prepend_size=True): for (offset, message) in items: encoded_values.append(Int64.encode(offset)) encoded_values.append(Bytes.encode(message)) - encoded = b''.join(encoded_values) + encoded = b"".join(encoded_values) if prepend_size: return Bytes.encode(encoded) else: diff --git a/aiokafka/protocol/metadata.py b/aiokafka/protocol/metadata.py new file mode 100644 index 00000000..79a5600a --- /dev/null +++ b/aiokafka/protocol/metadata.py @@ -0,0 +1,260 @@ +from .api import Request, Response +from .types import Array, Boolean, Int16, Int32, Schema, String + + +class MetadataResponse_v0(Response): + API_KEY = 3 + API_VERSION = 0 + SCHEMA = Schema( + ( + "brokers", + Array(("node_id", Int32), ("host", String("utf-8")), ("port", Int32)), + ), + ( + "topics", + Array( + ("error_code", Int16), + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("error_code", Int16), + ("partition", Int32), + ("leader", Int32), + ("replicas", Array(Int32)), + ("isr", Array(Int32)), + ), + ), + ), + ), + ) + + +class MetadataResponse_v1(Response): + API_KEY = 3 + API_VERSION = 1 + SCHEMA = Schema( + ( + "brokers", + Array( + ("node_id", Int32), + ("host", String("utf-8")), + ("port", Int32), + ("rack", String("utf-8")), + ), + ), + ("controller_id", Int32), + ( + "topics", + Array( + ("error_code", Int16), + ("topic", String("utf-8")), + ("is_internal", Boolean), + ( + "partitions", + Array( + ("error_code", Int16), + ("partition", Int32), + ("leader", Int32), + ("replicas", Array(Int32)), + ("isr", Array(Int32)), + ), + ), + ), + ), + ) + + +class MetadataResponse_v2(Response): + API_KEY = 3 + API_VERSION = 2 + SCHEMA = Schema( + ( + "brokers", + Array( + ("node_id", Int32), + ("host", String("utf-8")), + ("port", Int32), + ("rack", String("utf-8")), + ), + ), + ("cluster_id", String("utf-8")), # <-- Added cluster_id field in v2 + ("controller_id", Int32), + ( + "topics", + Array( + ("error_code", Int16), + ("topic", String("utf-8")), + ("is_internal", Boolean), + ( + "partitions", + Array( + ("error_code", Int16), + ("partition", Int32), + ("leader", Int32), + ("replicas", Array(Int32)), + ("isr", Array(Int32)), + ), + ), + ), + ), + ) + + +class MetadataResponse_v3(Response): + API_KEY = 3 + API_VERSION = 3 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "brokers", + Array( + ("node_id", Int32), + ("host", String("utf-8")), + ("port", Int32), + ("rack", String("utf-8")), + ), + ), + ("cluster_id", String("utf-8")), + ("controller_id", Int32), + ( + "topics", + Array( + ("error_code", Int16), + ("topic", String("utf-8")), + ("is_internal", Boolean), + ( + "partitions", + Array( + ("error_code", Int16), + ("partition", Int32), + ("leader", Int32), + ("replicas", Array(Int32)), + ("isr", Array(Int32)), + ), + ), + ), + ), + ) + + +class MetadataResponse_v4(Response): + API_KEY = 3 + API_VERSION = 4 + SCHEMA = MetadataResponse_v3.SCHEMA + + +class MetadataResponse_v5(Response): + API_KEY = 3 + API_VERSION = 5 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "brokers", + Array( + ("node_id", Int32), + ("host", String("utf-8")), + ("port", Int32), + ("rack", String("utf-8")), + ), + ), + ("cluster_id", String("utf-8")), + ("controller_id", Int32), + ( + "topics", + Array( + ("error_code", Int16), + ("topic", String("utf-8")), + ("is_internal", Boolean), + ( + "partitions", + Array( + ("error_code", Int16), + ("partition", Int32), + ("leader", Int32), + ("replicas", Array(Int32)), + ("isr", Array(Int32)), + ("offline_replicas", Array(Int32)), + ), + ), + ), + ), + ) + + +class MetadataRequest_v0(Request): + API_KEY = 3 + API_VERSION = 0 + RESPONSE_TYPE = MetadataResponse_v0 + SCHEMA = Schema(("topics", Array(String("utf-8")))) + ALL_TOPICS = None # Empty Array (len 0) for topics returns all topics + + +class MetadataRequest_v1(Request): + API_KEY = 3 + API_VERSION = 1 + RESPONSE_TYPE = MetadataResponse_v1 + SCHEMA = MetadataRequest_v0.SCHEMA + ALL_TOPICS = -1 # Null Array (len -1) for topics returns all topics + NO_TOPICS = None # Empty array (len 0) for topics returns no topics + + +class MetadataRequest_v2(Request): + API_KEY = 3 + API_VERSION = 2 + RESPONSE_TYPE = MetadataResponse_v2 + SCHEMA = MetadataRequest_v1.SCHEMA + ALL_TOPICS = -1 # Null Array (len -1) for topics returns all topics + NO_TOPICS = None # Empty array (len 0) for topics returns no topics + + +class MetadataRequest_v3(Request): + API_KEY = 3 + API_VERSION = 3 + RESPONSE_TYPE = MetadataResponse_v3 + SCHEMA = MetadataRequest_v1.SCHEMA + ALL_TOPICS = -1 # Null Array (len -1) for topics returns all topics + NO_TOPICS = None # Empty array (len 0) for topics returns no topics + + +class MetadataRequest_v4(Request): + API_KEY = 3 + API_VERSION = 4 + RESPONSE_TYPE = MetadataResponse_v4 + SCHEMA = Schema( + ("topics", Array(String("utf-8"))), ("allow_auto_topic_creation", Boolean) + ) + ALL_TOPICS = -1 # Null Array (len -1) for topics returns all topics + NO_TOPICS = None # Empty array (len 0) for topics returns no topics + + +class MetadataRequest_v5(Request): + """ + The v5 metadata request is the same as v4. + An additional field for offline_replicas has been added to the v5 metadata response + """ + + API_KEY = 3 + API_VERSION = 5 + RESPONSE_TYPE = MetadataResponse_v5 + SCHEMA = MetadataRequest_v4.SCHEMA + ALL_TOPICS = -1 # Null Array (len -1) for topics returns all topics + NO_TOPICS = None # Empty array (len 0) for topics returns no topics + + +MetadataRequest = [ + MetadataRequest_v0, + MetadataRequest_v1, + MetadataRequest_v2, + MetadataRequest_v3, + MetadataRequest_v4, + MetadataRequest_v5, +] +MetadataResponse = [ + MetadataResponse_v0, + MetadataResponse_v1, + MetadataResponse_v2, + MetadataResponse_v3, + MetadataResponse_v4, + MetadataResponse_v5, +] diff --git a/aiokafka/protocol/offset.py b/aiokafka/protocol/offset.py new file mode 100644 index 00000000..11eef1e8 --- /dev/null +++ b/aiokafka/protocol/offset.py @@ -0,0 +1,246 @@ +from .api import Request, Response +from .types import Array, Int8, Int16, Int32, Int64, Schema, String + +UNKNOWN_OFFSET = -1 + + +class OffsetResetStrategy(object): + LATEST = -1 + EARLIEST = -2 + NONE = 0 + + +class OffsetResponse_v0(Response): + API_KEY = 2 + API_VERSION = 0 + SCHEMA = Schema( + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("error_code", Int16), + ("offsets", Array(Int64)), + ), + ), + ), + ) + ) + + +class OffsetResponse_v1(Response): + API_KEY = 2 + API_VERSION = 1 + SCHEMA = Schema( + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("error_code", Int16), + ("timestamp", Int64), + ("offset", Int64), + ), + ), + ), + ) + ) + + +class OffsetResponse_v2(Response): + API_KEY = 2 + API_VERSION = 2 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("error_code", Int16), + ("timestamp", Int64), + ("offset", Int64), + ), + ), + ), + ), + ) + + +class OffsetResponse_v3(Response): + """ + on quota violation, brokers send out responses before throttling + """ + + API_KEY = 2 + API_VERSION = 3 + SCHEMA = OffsetResponse_v2.SCHEMA + + +class OffsetResponse_v4(Response): + """ + Add leader_epoch to response + """ + + API_KEY = 2 + API_VERSION = 4 + SCHEMA = Schema( + ("throttle_time_ms", Int32), + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("error_code", Int16), + ("timestamp", Int64), + ("offset", Int64), + ("leader_epoch", Int32), + ), + ), + ), + ), + ) + + +class OffsetResponse_v5(Response): + """ + adds a new error code, OFFSET_NOT_AVAILABLE + """ + + API_KEY = 2 + API_VERSION = 5 + SCHEMA = OffsetResponse_v4.SCHEMA + + +class OffsetRequest_v0(Request): + API_KEY = 2 + API_VERSION = 0 + RESPONSE_TYPE = OffsetResponse_v0 + SCHEMA = Schema( + ("replica_id", Int32), + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("timestamp", Int64), + ("max_offsets", Int32), + ), + ), + ), + ), + ) + DEFAULTS = {"replica_id": -1} + + +class OffsetRequest_v1(Request): + API_KEY = 2 + API_VERSION = 1 + RESPONSE_TYPE = OffsetResponse_v1 + SCHEMA = Schema( + ("replica_id", Int32), + ( + "topics", + Array( + ("topic", String("utf-8")), + ("partitions", Array(("partition", Int32), ("timestamp", Int64))), + ), + ), + ) + DEFAULTS = {"replica_id": -1} + + +class OffsetRequest_v2(Request): + API_KEY = 2 + API_VERSION = 2 + RESPONSE_TYPE = OffsetResponse_v2 + SCHEMA = Schema( + ("replica_id", Int32), + ("isolation_level", Int8), # <- added isolation_level + ( + "topics", + Array( + ("topic", String("utf-8")), + ("partitions", Array(("partition", Int32), ("timestamp", Int64))), + ), + ), + ) + DEFAULTS = {"replica_id": -1} + + +class OffsetRequest_v3(Request): + API_KEY = 2 + API_VERSION = 3 + RESPONSE_TYPE = OffsetResponse_v3 + SCHEMA = OffsetRequest_v2.SCHEMA + DEFAULTS = {"replica_id": -1} + + +class OffsetRequest_v4(Request): + """ + Add current_leader_epoch to request + """ + + API_KEY = 2 + API_VERSION = 4 + RESPONSE_TYPE = OffsetResponse_v4 + SCHEMA = Schema( + ("replica_id", Int32), + ("isolation_level", Int8), # <- added isolation_level + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("current_leader_epoch", Int64), + ("timestamp", Int64), + ), + ), + ), + ), + ) + DEFAULTS = {"replica_id": -1} + + +class OffsetRequest_v5(Request): + API_KEY = 2 + API_VERSION = 5 + RESPONSE_TYPE = OffsetResponse_v5 + SCHEMA = OffsetRequest_v4.SCHEMA + DEFAULTS = {"replica_id": -1} + + +OffsetRequest = [ + OffsetRequest_v0, + OffsetRequest_v1, + OffsetRequest_v2, + OffsetRequest_v3, + OffsetRequest_v4, + OffsetRequest_v5, +] +OffsetResponse = [ + OffsetResponse_v0, + OffsetResponse_v1, + OffsetResponse_v2, + OffsetResponse_v3, + OffsetResponse_v4, + OffsetResponse_v5, +] diff --git a/kafka/protocol/parser.py b/aiokafka/protocol/parser.py similarity index 69% rename from kafka/protocol/parser.py rename to aiokafka/protocol/parser.py index a872202d..c19dc4f1 100644 --- a/kafka/protocol/parser.py +++ b/aiokafka/protocol/parser.py @@ -1,13 +1,12 @@ -from __future__ import absolute_import - import collections import logging import aiokafka.errors as Errors -from kafka.protocol.commit import GroupCoordinatorResponse -from kafka.protocol.frame import KafkaBytes -from kafka.protocol.types import Int32, TaggedFields -from kafka.version import __version__ +from aiokafka import __version__ + +from .commit import GroupCoordinatorResponse +from .frame import KafkaBytes +from .types import Int32 log = logging.getLogger(__name__) @@ -24,6 +23,7 @@ class KafkaProtocol(object): Currently only used to check for 0.8.2 protocol quirks, but may be used for more in the future. """ + def __init__(self, client_id=None, api_version=None): if client_id is None: client_id = self._gen_client_id() @@ -41,7 +41,7 @@ def _next_correlation_id(self): return self._correlation_id def _gen_client_id(self): - return 'kafka-python' + __version__ + return "aiokafka" + __version__ def send_request(self, request, correlation_id=None): """Encode and queue a kafka api request for sending. @@ -55,12 +55,14 @@ def send_request(self, request, correlation_id=None): Returns: correlation_id """ - log.debug('Sending request %s', request) + log.debug("Sending request %s", request) if correlation_id is None: correlation_id = self._next_correlation_id() - header = request.build_request_header(correlation_id=correlation_id, client_id=self._client_id) - message = b''.join([header.encode(), request.encode()]) + header = request.build_request_header( + correlation_id=correlation_id, client_id=self._client_id + ) + message = b"".join([header.encode(), request.encode()]) size = Int32.encode(len(message)) data = size + message self.bytes_to_send.append(data) @@ -71,7 +73,7 @@ def send_request(self, request, correlation_id=None): def send_bytes(self): """Retrieve all pending bytes to send on the network""" - data = b''.join(self.bytes_to_send) + data = b"".join(self.bytes_to_send) self.bytes_to_send = [] return data @@ -99,7 +101,7 @@ def receive_bytes(self, data): # Not receiving is the state of reading the payload header if not self._receiving: bytes_to_read = min(4 - self._header.tell(), n - i) - self._header.write(data[i:i+bytes_to_read]) + self._header.write(data[i:i + bytes_to_read]) i += bytes_to_read if self._header.tell() == 4: @@ -109,18 +111,22 @@ def receive_bytes(self, data): self._rbuffer = KafkaBytes(nbytes) self._receiving = True elif self._header.tell() > 4: - raise Errors.KafkaError('this should not happen - are you threading?') + raise Errors.KafkaError( + "this should not happen - are you threading?" + ) if self._receiving: total_bytes = len(self._rbuffer) staged_bytes = self._rbuffer.tell() bytes_to_read = min(total_bytes - staged_bytes, n - i) - self._rbuffer.write(data[i:i+bytes_to_read]) + self._rbuffer.write(data[i:i + bytes_to_read]) i += bytes_to_read staged_bytes = self._rbuffer.tell() if staged_bytes > total_bytes: - raise Errors.KafkaError('Receive buffer has more bytes than expected?') + raise Errors.KafkaError( + "Receive buffer has more bytes than expected?" + ) if staged_bytes != total_bytes: break @@ -134,39 +140,51 @@ def receive_bytes(self, data): def _process_response(self, read_buffer): if not self.in_flight_requests: - raise Errors.CorrelationIdError('No in-flight-request found for server response') + raise Errors.CorrelationIdError( + "No in-flight-request found for server response" + ) (correlation_id, request) = self.in_flight_requests.popleft() response_header = request.parse_response_header(read_buffer) recv_correlation_id = response_header.correlation_id - log.debug('Received correlation id: %d', recv_correlation_id) + log.debug("Received correlation id: %d", recv_correlation_id) # 0.8.2 quirk - if (recv_correlation_id == 0 and - correlation_id != 0 and - request.RESPONSE_TYPE is GroupCoordinatorResponse[0] and - (self._api_version == (0, 8, 2) or self._api_version is None)): - log.warning('Kafka 0.8.2 quirk -- GroupCoordinatorResponse' - ' Correlation ID does not match request. This' - ' should go away once at least one topic has been' - ' initialized on the broker.') + if ( + recv_correlation_id == 0 + and correlation_id != 0 + and request.RESPONSE_TYPE is GroupCoordinatorResponse[0] + and (self._api_version == (0, 8, 2) or self._api_version is None) + ): + log.warning( + "Kafka 0.8.2 quirk -- GroupCoordinatorResponse" + " Correlation ID does not match request. This" + " should go away once at least one topic has been" + " initialized on the broker." + ) elif correlation_id != recv_correlation_id: # return or raise? raise Errors.CorrelationIdError( - 'Correlation IDs do not match: sent %d, recv %d' - % (correlation_id, recv_correlation_id)) + "Correlation IDs do not match: sent %d, recv %d" + % (correlation_id, recv_correlation_id) + ) # decode response - log.debug('Processing response %s', request.RESPONSE_TYPE.__name__) + log.debug("Processing response %s", request.RESPONSE_TYPE.__name__) try: response = request.RESPONSE_TYPE.decode(read_buffer) except ValueError: read_buffer.seek(0) buf = read_buffer.read() - log.error('Response %d [ResponseType: %s Request: %s]:' - ' Unable to decode %d-byte buffer: %r', - correlation_id, request.RESPONSE_TYPE, - request, len(buf), buf) - raise Errors.KafkaProtocolError('Unable to decode response') + log.error( + "Response %d [ResponseType: %s Request: %s]:" + " Unable to decode %d-byte buffer: %r", + correlation_id, + request.RESPONSE_TYPE, + request, + len(buf), + buf, + ) + raise Errors.KafkaProtocolError("Unable to decode response") return (correlation_id, response) diff --git a/kafka/protocol/pickle.py b/aiokafka/protocol/pickle.py similarity index 80% rename from kafka/protocol/pickle.py rename to aiokafka/protocol/pickle.py index d6e5fa74..780c4e88 100644 --- a/kafka/protocol/pickle.py +++ b/aiokafka/protocol/pickle.py @@ -1,10 +1,4 @@ -from __future__ import absolute_import - -try: - import copyreg # pylint: disable=import-error -except ImportError: - import copy_reg as copyreg # pylint: disable=import-error - +import copyreg import types @@ -31,5 +25,6 @@ def _unpickle_method(func_name, obj, cls): break return func.__get__(obj, cls) + # https://bytes.com/topic/python/answers/552476-why-cant-you-pickle-instancemethods copyreg.pickle(types.MethodType, _pickle_method, _unpickle_method) diff --git a/aiokafka/protocol/produce.py b/aiokafka/protocol/produce.py new file mode 100644 index 00000000..4083b941 --- /dev/null +++ b/aiokafka/protocol/produce.py @@ -0,0 +1,299 @@ +from .api import Request, Response +from .types import Int16, Int32, Int64, String, Array, Schema, Bytes + + +class ProduceResponse_v0(Response): + API_KEY = 0 + API_VERSION = 0 + SCHEMA = Schema( + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), ("error_code", Int16), ("offset", Int64) + ), + ), + ), + ) + ) + + +class ProduceResponse_v1(Response): + API_KEY = 0 + API_VERSION = 1 + SCHEMA = Schema( + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), ("error_code", Int16), ("offset", Int64) + ), + ), + ), + ), + ("throttle_time_ms", Int32), + ) + + +class ProduceResponse_v2(Response): + API_KEY = 0 + API_VERSION = 2 + SCHEMA = Schema( + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("error_code", Int16), + ("offset", Int64), + ("timestamp", Int64), + ), + ), + ), + ), + ("throttle_time_ms", Int32), + ) + + +class ProduceResponse_v3(Response): + API_KEY = 0 + API_VERSION = 3 + SCHEMA = ProduceResponse_v2.SCHEMA + + +class ProduceResponse_v4(Response): + """ + The version number is bumped up to indicate that the client supports + KafkaStorageException. The KafkaStorageException will be translated to + NotLeaderForPartitionException in the response if version <= 3 + """ + + API_KEY = 0 + API_VERSION = 4 + SCHEMA = ProduceResponse_v3.SCHEMA + + +class ProduceResponse_v5(Response): + API_KEY = 0 + API_VERSION = 5 + SCHEMA = Schema( + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("error_code", Int16), + ("offset", Int64), + ("timestamp", Int64), + ("log_start_offset", Int64), + ), + ), + ), + ), + ("throttle_time_ms", Int32), + ) + + +class ProduceResponse_v6(Response): + """ + The version number is bumped to indicate that on quota violation brokers send out + responses before throttling. + """ + + API_KEY = 0 + API_VERSION = 6 + SCHEMA = ProduceResponse_v5.SCHEMA + + +class ProduceResponse_v7(Response): + """ + V7 bumped up to indicate ZStandard capability. (see KIP-110) + """ + + API_KEY = 0 + API_VERSION = 7 + SCHEMA = ProduceResponse_v6.SCHEMA + + +class ProduceResponse_v8(Response): + """ + V8 bumped up to add two new fields record_errors offset list and error_message + (See KIP-467) + """ + + API_KEY = 0 + API_VERSION = 8 + SCHEMA = Schema( + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("error_code", Int16), + ("offset", Int64), + ("timestamp", Int64), + ("log_start_offset", Int64), + ), + ( + "record_errors", + ( + Array( + ("batch_index", Int32), + ("batch_index_error_message", String("utf-8")), + ) + ), + ), + ("error_message", String("utf-8")), + ), + ), + ), + ("throttle_time_ms", Int32), + ) + + +class ProduceRequest(Request): + API_KEY = 0 + + def expect_response(self): + if self.required_acks == 0: # pylint: disable=no-member + return False + return True + + +class ProduceRequest_v0(ProduceRequest): + API_VERSION = 0 + RESPONSE_TYPE = ProduceResponse_v0 + SCHEMA = Schema( + ("required_acks", Int16), + ("timeout", Int32), + ( + "topics", + Array( + ("topic", String("utf-8")), + ("partitions", Array(("partition", Int32), ("messages", Bytes))), + ), + ), + ) + + +class ProduceRequest_v1(ProduceRequest): + API_VERSION = 1 + RESPONSE_TYPE = ProduceResponse_v1 + SCHEMA = ProduceRequest_v0.SCHEMA + + +class ProduceRequest_v2(ProduceRequest): + API_VERSION = 2 + RESPONSE_TYPE = ProduceResponse_v2 + SCHEMA = ProduceRequest_v1.SCHEMA + + +class ProduceRequest_v3(ProduceRequest): + API_VERSION = 3 + RESPONSE_TYPE = ProduceResponse_v3 + SCHEMA = Schema( + ("transactional_id", String("utf-8")), + ("required_acks", Int16), + ("timeout", Int32), + ( + "topics", + Array( + ("topic", String("utf-8")), + ("partitions", Array(("partition", Int32), ("messages", Bytes))), + ), + ), + ) + + +class ProduceRequest_v4(ProduceRequest): + """ + The version number is bumped up to indicate that the client supports + KafkaStorageException. The KafkaStorageException will be translated to + NotLeaderForPartitionException in the response if version <= 3 + """ + + API_VERSION = 4 + RESPONSE_TYPE = ProduceResponse_v4 + SCHEMA = ProduceRequest_v3.SCHEMA + + +class ProduceRequest_v5(ProduceRequest): + """ + Same as v4. The version number is bumped since the v5 response includes an + additional partition level field: the log_start_offset. + """ + + API_VERSION = 5 + RESPONSE_TYPE = ProduceResponse_v5 + SCHEMA = ProduceRequest_v4.SCHEMA + + +class ProduceRequest_v6(ProduceRequest): + """ + The version number is bumped to indicate that on quota violation brokers send out + responses before throttling. + """ + + API_VERSION = 6 + RESPONSE_TYPE = ProduceResponse_v6 + SCHEMA = ProduceRequest_v5.SCHEMA + + +class ProduceRequest_v7(ProduceRequest): + """ + V7 bumped up to indicate ZStandard capability. (see KIP-110) + """ + + API_VERSION = 7 + RESPONSE_TYPE = ProduceResponse_v7 + SCHEMA = ProduceRequest_v6.SCHEMA + + +class ProduceRequest_v8(ProduceRequest): + """ + V8 bumped up to add two new fields record_errors offset list and error_message to + PartitionResponse (See KIP-467) + """ + + API_VERSION = 8 + RESPONSE_TYPE = ProduceResponse_v8 + SCHEMA = ProduceRequest_v7.SCHEMA + + +ProduceRequest = [ + ProduceRequest_v0, + ProduceRequest_v1, + ProduceRequest_v2, + ProduceRequest_v3, + ProduceRequest_v4, + ProduceRequest_v5, + ProduceRequest_v6, + ProduceRequest_v7, + ProduceRequest_v8, +] +ProduceResponse = [ + ProduceResponse_v0, + ProduceResponse_v1, + ProduceResponse_v2, + ProduceResponse_v3, + ProduceResponse_v4, + ProduceResponse_v5, + ProduceResponse_v6, + ProduceResponse_v7, + ProduceResponse_v8, +] diff --git a/kafka/protocol/struct.py b/aiokafka/protocol/struct.py similarity index 72% rename from kafka/protocol/struct.py rename to aiokafka/protocol/struct.py index e9da6e6c..d7faa327 100644 --- a/kafka/protocol/struct.py +++ b/aiokafka/protocol/struct.py @@ -1,9 +1,7 @@ -from __future__ import absolute_import - from io import BytesIO -from kafka.protocol.abstract import AbstractType -from kafka.protocol.types import Schema +from .abstract import AbstractType +from .types import Schema from kafka.util import WeakMethod @@ -16,32 +14,30 @@ def __init__(self, *args, **kwargs): for i, name in enumerate(self.SCHEMA.names): self.__dict__[name] = args[i] elif len(args) > 0: - raise ValueError('Args must be empty or mirror schema') + raise ValueError("Args must be empty or mirror schema") else: for name in self.SCHEMA.names: self.__dict__[name] = kwargs.pop(name, None) if kwargs: - raise ValueError('Keyword(s) not in schema %s: %s' - % (list(self.SCHEMA.names), - ', '.join(kwargs.keys()))) + raise ValueError( + "Keyword(s) not in schema %s: %s" + % (list(self.SCHEMA.names), ", ".join(kwargs.keys())) + ) # overloading encode() to support both class and instance # Without WeakMethod() this creates circular ref, which # causes instances to "leak" to garbage self.encode = WeakMethod(self._encode_self) - @classmethod def encode(cls, item): # pylint: disable=E0202 bits = [] for i, field in enumerate(cls.SCHEMA.fields): bits.append(field.encode(item[i])) - return b''.join(bits) + return b"".join(bits) def _encode_self(self): - return self.SCHEMA.encode( - [self.__dict__[name] for name in self.SCHEMA.names] - ) + return self.SCHEMA.encode([self.__dict__[name] for name in self.SCHEMA.names]) @classmethod def decode(cls, data): @@ -57,8 +53,8 @@ def get_item(self, name): def __repr__(self): key_vals = [] for name, field in zip(self.SCHEMA.names, self.SCHEMA.fields): - key_vals.append('%s=%s' % (name, field.repr(self.__dict__[name]))) - return self.__class__.__name__ + '(' + ', '.join(key_vals) + ')' + key_vals.append("%s=%s" % (name, field.repr(self.__dict__[name]))) + return self.__class__.__name__ + "(" + ", ".join(key_vals) + ")" def __hash__(self): return hash(self.encode()) diff --git a/aiokafka/protocol/transaction.py b/aiokafka/protocol/transaction.py index c684bf3d..aa05d22d 100644 --- a/aiokafka/protocol/transaction.py +++ b/aiokafka/protocol/transaction.py @@ -1,17 +1,15 @@ -from kafka.protocol.api import Request, Response -from kafka.protocol.types import ( - Int16, Int32, Int64, Schema, String, Array, Boolean -) +from .api import Request, Response +from .types import Int16, Int32, Int64, Schema, String, Array, Boolean class InitProducerIdResponse_v0(Response): API_KEY = 22 API_VERSION = 0 SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('error_code', Int16), - ('producer_id', Int64), - ('producer_epoch', Int16), + ("throttle_time_ms", Int32), + ("error_code", Int16), + ("producer_id", Int64), + ("producer_epoch", Int16), ) @@ -20,8 +18,7 @@ class InitProducerIdRequest_v0(Request): API_VERSION = 0 RESPONSE_TYPE = InitProducerIdResponse_v0 SCHEMA = Schema( - ('transactional_id', String('utf-8')), - ('transaction_timeout_ms', Int32) + ("transactional_id", String("utf-8")), ("transaction_timeout_ms", Int32) ) @@ -29,12 +26,17 @@ class AddPartitionsToTxnResponse_v0(Response): API_KEY = 24 API_VERSION = 0 SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('errors', Array( - ('topic', String('utf-8')), - ('partition_errors', Array( - ('partition', Int32), - ('error_code', Int16))))) + ("throttle_time_ms", Int32), + ( + "errors", + Array( + ("topic", String("utf-8")), + ( + "partition_errors", + Array(("partition", Int32), ("error_code", Int16)), + ), + ), + ), ) @@ -43,22 +45,17 @@ class AddPartitionsToTxnRequest_v0(Request): API_VERSION = 0 RESPONSE_TYPE = AddPartitionsToTxnResponse_v0 SCHEMA = Schema( - ('transactional_id', String('utf-8')), - ('producer_id', Int64), - ('producer_epoch', Int16), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array(Int32)))) + ("transactional_id", String("utf-8")), + ("producer_id", Int64), + ("producer_epoch", Int16), + ("topics", Array(("topic", String("utf-8")), ("partitions", Array(Int32)))), ) class AddOffsetsToTxnResponse_v0(Response): API_KEY = 25 API_VERSION = 0 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('error_code', Int16) - ) + SCHEMA = Schema(("throttle_time_ms", Int32), ("error_code", Int16)) class AddOffsetsToTxnRequest_v0(Request): @@ -66,20 +63,17 @@ class AddOffsetsToTxnRequest_v0(Request): API_VERSION = 0 RESPONSE_TYPE = AddOffsetsToTxnResponse_v0 SCHEMA = Schema( - ('transactional_id', String('utf-8')), - ('producer_id', Int64), - ('producer_epoch', Int16), - ('group_id', String('utf-8')) + ("transactional_id", String("utf-8")), + ("producer_id", Int64), + ("producer_epoch", Int16), + ("group_id", String("utf-8")), ) class EndTxnResponse_v0(Response): API_KEY = 26 API_VERSION = 0 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('error_code', Int16) - ) + SCHEMA = Schema(("throttle_time_ms", Int32), ("error_code", Int16)) class EndTxnRequest_v0(Request): @@ -87,10 +81,10 @@ class EndTxnRequest_v0(Request): API_VERSION = 0 RESPONSE_TYPE = EndTxnResponse_v0 SCHEMA = Schema( - ('transactional_id', String('utf-8')), - ('producer_id', Int64), - ('producer_epoch', Int16), - ('transaction_result', Boolean) + ("transactional_id", String("utf-8")), + ("producer_id", Int64), + ("producer_epoch", Int16), + ("transaction_result", Boolean), ) @@ -98,12 +92,17 @@ class TxnOffsetCommitResponse_v0(Response): API_KEY = 28 API_VERSION = 0 SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('errors', Array( - ('topic', String('utf-8')), - ('partition_errors', Array( - ('partition', Int32), - ('error_code', Int16))))) + ("throttle_time_ms", Int32), + ( + "errors", + Array( + ("topic", String("utf-8")), + ( + "partition_errors", + Array(("partition", Int32), ("error_code", Int16)), + ), + ), + ), ) @@ -112,52 +111,40 @@ class TxnOffsetCommitRequest_v0(Request): API_VERSION = 0 RESPONSE_TYPE = TxnOffsetCommitResponse_v0 SCHEMA = Schema( - ('transactional_id', String('utf-8')), - ('group_id', String('utf-8')), - ('producer_id', Int64), - ('producer_epoch', Int16), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('offset', Int64), - ('metadata', String('utf-8')))))) + ("transactional_id", String("utf-8")), + ("group_id", String("utf-8")), + ("producer_id", Int64), + ("producer_epoch", Int16), + ( + "topics", + Array( + ("topic", String("utf-8")), + ( + "partitions", + Array( + ("partition", Int32), + ("offset", Int64), + ("metadata", String("utf-8")), + ), + ), + ), + ), ) -InitProducerIdRequest = [ - InitProducerIdRequest_v0 -] -InitProducerIdResponse = [ - InitProducerIdResponse_v0 -] - -AddPartitionsToTxnRequest = [ - AddPartitionsToTxnRequest_v0 -] -AddPartitionsToTxnResponse = [ - AddPartitionsToTxnResponse_v0 -] - -AddOffsetsToTxnRequest = [ - AddOffsetsToTxnRequest_v0 -] -AddOffsetsToTxnResponse = [ - AddOffsetsToTxnResponse_v0 -] - -EndTxnRequest = [ - EndTxnRequest_v0 -] - -EndTxnResponse = [ - EndTxnResponse_v0 -] - -TxnOffsetCommitResponse = [ - TxnOffsetCommitResponse_v0 -] - -TxnOffsetCommitRequest = [ - TxnOffsetCommitRequest_v0 -] +InitProducerIdRequest = [InitProducerIdRequest_v0] +InitProducerIdResponse = [InitProducerIdResponse_v0] + +AddPartitionsToTxnRequest = [AddPartitionsToTxnRequest_v0] +AddPartitionsToTxnResponse = [AddPartitionsToTxnResponse_v0] + +AddOffsetsToTxnRequest = [AddOffsetsToTxnRequest_v0] +AddOffsetsToTxnResponse = [AddOffsetsToTxnResponse_v0] + +EndTxnRequest = [EndTxnRequest_v0] + +EndTxnResponse = [EndTxnResponse_v0] + +TxnOffsetCommitResponse = [TxnOffsetCommitResponse_v0] + +TxnOffsetCommitRequest = [TxnOffsetCommitRequest_v0] diff --git a/kafka/protocol/types.py b/aiokafka/protocol/types.py similarity index 68% rename from kafka/protocol/types.py rename to aiokafka/protocol/types.py index 0e3685d7..56613905 100644 --- a/kafka/protocol/types.py +++ b/aiokafka/protocol/types.py @@ -1,18 +1,17 @@ -from __future__ import absolute_import - import struct from struct import error -from kafka.protocol.abstract import AbstractType +from .abstract import AbstractType def _pack(f, value): try: return f(value) except error as e: - raise ValueError("Error encountered when attempting to convert value: " - "{!r} to struct format: '{}', hit error: {}" - .format(value, f, e)) + raise ValueError( + "Error encountered when attempting to convert value: " + "{!r} to struct format: '{}', hit error: {}".format(value, f, e) + ) def _unpack(f, data): @@ -20,14 +19,15 @@ def _unpack(f, data): (value,) = f(data) return value except error as e: - raise ValueError("Error encountered when attempting to convert value: " - "{!r} to struct format: '{}', hit error: {}" - .format(data, f, e)) + raise ValueError( + "Error encountered when attempting to convert value: " + "{!r} to struct format: '{}', hit error: {}".format(data, f, e) + ) class Int8(AbstractType): - _pack = struct.Struct('>b').pack - _unpack = struct.Struct('>b').unpack + _pack = struct.Struct(">b").pack + _unpack = struct.Struct(">b").unpack @classmethod def encode(cls, value): @@ -39,8 +39,8 @@ def decode(cls, data): class Int16(AbstractType): - _pack = struct.Struct('>h').pack - _unpack = struct.Struct('>h').unpack + _pack = struct.Struct(">h").pack + _unpack = struct.Struct(">h").unpack @classmethod def encode(cls, value): @@ -52,8 +52,8 @@ def decode(cls, data): class Int32(AbstractType): - _pack = struct.Struct('>i').pack - _unpack = struct.Struct('>i').unpack + _pack = struct.Struct(">i").pack + _unpack = struct.Struct(">i").unpack @classmethod def encode(cls, value): @@ -65,8 +65,8 @@ def decode(cls, data): class Int64(AbstractType): - _pack = struct.Struct('>q').pack - _unpack = struct.Struct('>q').unpack + _pack = struct.Struct(">q").pack + _unpack = struct.Struct(">q").unpack @classmethod def encode(cls, value): @@ -78,8 +78,8 @@ def decode(cls, data): class Float64(AbstractType): - _pack = struct.Struct('>d').pack - _unpack = struct.Struct('>d').unpack + _pack = struct.Struct(">d").pack + _unpack = struct.Struct(">d").unpack @classmethod def encode(cls, value): @@ -91,7 +91,7 @@ def decode(cls, data): class String(AbstractType): - def __init__(self, encoding='utf-8'): + def __init__(self, encoding="utf-8"): self.encoding = encoding def encode(self, value): @@ -106,7 +106,7 @@ def decode(self, data): return None value = data.read(length) if len(value) != length: - raise ValueError('Buffer underrun decoding string') + raise ValueError("Buffer underrun decoding string") return value.decode(self.encoding) @@ -125,17 +125,19 @@ def decode(cls, data): return None value = data.read(length) if len(value) != length: - raise ValueError('Buffer underrun decoding Bytes') + raise ValueError("Buffer underrun decoding Bytes") return value @classmethod def repr(cls, value): - return repr(value[:100] + b'...' if value is not None and len(value) > 100 else value) + return repr( + value[:100] + b"..." if value is not None and len(value) > 100 else value + ) class Boolean(AbstractType): - _pack = struct.Struct('>?').pack - _unpack = struct.Struct('>?').unpack + _pack = struct.Struct(">?").pack + _unpack = struct.Struct(">?").unpack @classmethod def encode(cls, value): @@ -155,11 +157,8 @@ def __init__(self, *fields): def encode(self, item): if len(item) != len(self.fields): - raise ValueError('Item field count does not match Schema') - return b''.join([ - field.encode(item[i]) - for i, field in enumerate(self.fields) - ]) + raise ValueError("Item field count does not match Schema") + return b"".join([field.encode(item[i]) for i, field in enumerate(self.fields)]) def decode(self, data): return tuple([field.decode(data) for field in self.fields]) @@ -175,8 +174,10 @@ def repr(self, value): field_val = getattr(value, self.names[i]) except AttributeError: field_val = value[i] - key_vals.append('%s=%s' % (self.names[i], self.fields[i].repr(field_val))) - return '(' + ', '.join(key_vals) + ')' + key_vals.append( + "%s=%s" % (self.names[i], self.fields[i].repr(field_val)) + ) + return "(" + ", ".join(key_vals) + ")" except Exception: return repr(value) @@ -185,20 +186,19 @@ class Array(AbstractType): def __init__(self, *array_of): if len(array_of) > 1: self.array_of = Schema(*array_of) - elif len(array_of) == 1 and (isinstance(array_of[0], AbstractType) or - issubclass(array_of[0], AbstractType)): + elif len(array_of) == 1 and ( + isinstance(array_of[0], AbstractType) + or issubclass(array_of[0], AbstractType) + ): self.array_of = array_of[0] else: - raise ValueError('Array instantiated with no array_of type') + raise ValueError("Array instantiated with no array_of type") def encode(self, items): if items is None: return Int32.encode(-1) encoded_items = [self.array_of.encode(item) for item in items] - return b''.join( - [Int32.encode(len(encoded_items))] + - encoded_items - ) + return b"".join([Int32.encode(len(encoded_items))] + encoded_items) def decode(self, data): length = Int32.decode(data) @@ -208,8 +208,10 @@ def decode(self, data): def repr(self, list_of_items): if list_of_items is None: - return 'NULL' - return '[' + ', '.join([self.array_of.repr(item) for item in list_of_items]) + ']' + return "NULL" + return ( + "[" + ", ".join([self.array_of.repr(item) for item in list_of_items]) + "]" + ) class UnsignedVarInt32(AbstractType): @@ -217,25 +219,25 @@ class UnsignedVarInt32(AbstractType): def decode(cls, data): value, i = 0, 0 while True: - b, = struct.unpack('B', data.read(1)) + (b,) = struct.unpack("B", data.read(1)) if not (b & 0x80): break - value |= (b & 0x7f) << i + value |= (b & 0x7F) << i i += 7 if i > 28: - raise ValueError('Invalid value {}'.format(value)) + raise ValueError("Invalid value {}".format(value)) value |= b << i return value @classmethod def encode(cls, value): - value &= 0xffffffff - ret = b'' - while (value & 0xffffff80) != 0: - b = (value & 0x7f) | 0x80 - ret += struct.pack('B', b) + value &= 0xFFFFFFFF + ret = b"" + while (value & 0xFFFFFF80) != 0: + b = (value & 0x7F) | 0x80 + ret += struct.pack("B", b) value >>= 7 - ret += struct.pack('B', value) + ret += struct.pack("B", value) return ret @@ -248,7 +250,7 @@ def decode(cls, data): @classmethod def encode(cls, value): # bring it in line with the java binary repr - value &= 0xffffffff + value &= 0xFFFFFFFF return UnsignedVarInt32.encode((value << 1) ^ (value >> 31)) @@ -260,24 +262,24 @@ def decode(cls, data): b = data.read(1) if not (b & 0x80): break - value |= (b & 0x7f) << i + value |= (b & 0x7F) << i i += 7 if i > 63: - raise ValueError('Invalid value {}'.format(value)) + raise ValueError("Invalid value {}".format(value)) value |= b << i return (value >> 1) ^ -(value & 1) @classmethod def encode(cls, value): # bring it in line with the java binary repr - value &= 0xffffffffffffffff + value &= 0xFFFFFFFFFFFFFFFF v = (value << 1) ^ (value >> 63) - ret = b'' - while (v & 0xffffffffffffff80) != 0: - b = (value & 0x7f) | 0x80 - ret += struct.pack('B', b) + ret = b"" + while (v & 0xFFFFFFFFFFFFFF80) != 0: + b = (value & 0x7F) | 0x80 + ret += struct.pack("B", b) v >>= 7 - ret += struct.pack('B', v) + ret += struct.pack("B", v) return ret @@ -288,7 +290,7 @@ def decode(self, data): return None value = data.read(length) if len(value) != length: - raise ValueError('Buffer underrun decoding string') + raise ValueError("Buffer underrun decoding string") return value.decode(self.encoding) def encode(self, value): @@ -309,7 +311,7 @@ def decode(cls, data): for i in range(num_fields): tag = UnsignedVarInt32.decode(data) if tag <= prev_tag: - raise ValueError('Invalid or out-of-order tag {}'.format(tag)) + raise ValueError("Invalid or out-of-order tag {}".format(tag)) prev_tag = tag size = UnsignedVarInt32.decode(data) val = data.read(size) @@ -321,8 +323,10 @@ def encode(cls, value): ret = UnsignedVarInt32.encode(len(value)) for k, v in value.items(): # do we allow for other data types ?? It could get complicated really fast - assert isinstance(v, bytes), 'Value {} is not a byte array'.format(v) - assert isinstance(k, int) and k > 0, 'Key {} is not a positive integer'.format(k) + assert isinstance(v, bytes), "Value {} is not a byte array".format(v) + assert ( + isinstance(k, int) and k > 0 + ), "Key {} is not a positive integer".format(k) ret += UnsignedVarInt32.encode(k) ret += v return ret @@ -336,7 +340,7 @@ def decode(cls, data): return None value = data.read(length) if len(value) != length: - raise ValueError('Buffer underrun decoding Bytes') + raise ValueError("Buffer underrun decoding Bytes") return value @classmethod @@ -348,13 +352,12 @@ def encode(cls, value): class CompactArray(Array): - def encode(self, items): if items is None: return UnsignedVarInt32.encode(0) - return b''.join( - [UnsignedVarInt32.encode(len(items) + 1)] + - [self.array_of.encode(item) for item in items] + return b"".join( + [UnsignedVarInt32.encode(len(items) + 1)] + + [self.array_of.encode(item) for item in items] ) def decode(self, data): @@ -362,4 +365,3 @@ def decode(self, data): if length == -1: return None return [self.array_of.decode(data) for _ in range(length)] - diff --git a/docs/api.rst b/docs/api.rst index ba75a088..9ca4f590 100644 --- a/docs/api.rst +++ b/docs/api.rst @@ -176,6 +176,6 @@ Structs Protocols ^^^^^^^^^ -.. autoclass:: kafka.protocol.produce.ProduceRequest +.. autoclass:: aiokafka.protocol.produce.ProduceRequest :member-order: alphabetical :members: diff --git a/kafka/__init__.py b/kafka/__init__.py index 16d29b2e..a40686e6 100644 --- a/kafka/__init__.py +++ b/kafka/__init__.py @@ -1,7 +1,6 @@ from __future__ import absolute_import __title__ = 'kafka' -from kafka.version import __version__ __author__ = 'Dana Powers' __license__ = 'Apache License 2.0' __copyright__ = 'Copyright 2016 Dana Powers, David Arthur, and Contributors' diff --git a/kafka/protocol/__init__.py b/kafka/protocol/__init__.py deleted file mode 100644 index 025447f9..00000000 --- a/kafka/protocol/__init__.py +++ /dev/null @@ -1,49 +0,0 @@ -from __future__ import absolute_import - - -API_KEYS = { - 0: 'Produce', - 1: 'Fetch', - 2: 'ListOffsets', - 3: 'Metadata', - 4: 'LeaderAndIsr', - 5: 'StopReplica', - 6: 'UpdateMetadata', - 7: 'ControlledShutdown', - 8: 'OffsetCommit', - 9: 'OffsetFetch', - 10: 'FindCoordinator', - 11: 'JoinGroup', - 12: 'Heartbeat', - 13: 'LeaveGroup', - 14: 'SyncGroup', - 15: 'DescribeGroups', - 16: 'ListGroups', - 17: 'SaslHandshake', - 18: 'ApiVersions', - 19: 'CreateTopics', - 20: 'DeleteTopics', - 21: 'DeleteRecords', - 22: 'InitProducerId', - 23: 'OffsetForLeaderEpoch', - 24: 'AddPartitionsToTxn', - 25: 'AddOffsetsToTxn', - 26: 'EndTxn', - 27: 'WriteTxnMarkers', - 28: 'TxnOffsetCommit', - 29: 'DescribeAcls', - 30: 'CreateAcls', - 31: 'DeleteAcls', - 32: 'DescribeConfigs', - 33: 'AlterConfigs', - 36: 'SaslAuthenticate', - 37: 'CreatePartitions', - 38: 'CreateDelegationToken', - 39: 'RenewDelegationToken', - 40: 'ExpireDelegationToken', - 41: 'DescribeDelegationToken', - 42: 'DeleteGroups', - 45: 'AlterPartitionReassignments', - 46: 'ListPartitionReassignments', - 48: 'DescribeClientQuotas', -} diff --git a/kafka/protocol/admin.py b/kafka/protocol/admin.py deleted file mode 100644 index f9d61e5c..00000000 --- a/kafka/protocol/admin.py +++ /dev/null @@ -1,1054 +0,0 @@ -from __future__ import absolute_import - -from kafka.protocol.api import Request, Response -from kafka.protocol.types import Array, Boolean, Bytes, Int8, Int16, Int32, Int64, Schema, String, Float64, CompactString, CompactArray, TaggedFields - - -class ApiVersionResponse_v0(Response): - API_KEY = 18 - API_VERSION = 0 - SCHEMA = Schema( - ('error_code', Int16), - ('api_versions', Array( - ('api_key', Int16), - ('min_version', Int16), - ('max_version', Int16))) - ) - - -class ApiVersionResponse_v1(Response): - API_KEY = 18 - API_VERSION = 1 - SCHEMA = Schema( - ('error_code', Int16), - ('api_versions', Array( - ('api_key', Int16), - ('min_version', Int16), - ('max_version', Int16))), - ('throttle_time_ms', Int32) - ) - - -class ApiVersionResponse_v2(Response): - API_KEY = 18 - API_VERSION = 2 - SCHEMA = ApiVersionResponse_v1.SCHEMA - - -class ApiVersionRequest_v0(Request): - API_KEY = 18 - API_VERSION = 0 - RESPONSE_TYPE = ApiVersionResponse_v0 - SCHEMA = Schema() - - -class ApiVersionRequest_v1(Request): - API_KEY = 18 - API_VERSION = 1 - RESPONSE_TYPE = ApiVersionResponse_v1 - SCHEMA = ApiVersionRequest_v0.SCHEMA - - -class ApiVersionRequest_v2(Request): - API_KEY = 18 - API_VERSION = 2 - RESPONSE_TYPE = ApiVersionResponse_v1 - SCHEMA = ApiVersionRequest_v0.SCHEMA - - -ApiVersionRequest = [ - ApiVersionRequest_v0, ApiVersionRequest_v1, ApiVersionRequest_v2, -] -ApiVersionResponse = [ - ApiVersionResponse_v0, ApiVersionResponse_v1, ApiVersionResponse_v2, -] - - -class CreateTopicsResponse_v0(Response): - API_KEY = 19 - API_VERSION = 0 - SCHEMA = Schema( - ('topic_errors', Array( - ('topic', String('utf-8')), - ('error_code', Int16))) - ) - - -class CreateTopicsResponse_v1(Response): - API_KEY = 19 - API_VERSION = 1 - SCHEMA = Schema( - ('topic_errors', Array( - ('topic', String('utf-8')), - ('error_code', Int16), - ('error_message', String('utf-8')))) - ) - - -class CreateTopicsResponse_v2(Response): - API_KEY = 19 - API_VERSION = 2 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('topic_errors', Array( - ('topic', String('utf-8')), - ('error_code', Int16), - ('error_message', String('utf-8')))) - ) - -class CreateTopicsResponse_v3(Response): - API_KEY = 19 - API_VERSION = 3 - SCHEMA = CreateTopicsResponse_v2.SCHEMA - - -class CreateTopicsRequest_v0(Request): - API_KEY = 19 - API_VERSION = 0 - RESPONSE_TYPE = CreateTopicsResponse_v0 - SCHEMA = Schema( - ('create_topic_requests', Array( - ('topic', String('utf-8')), - ('num_partitions', Int32), - ('replication_factor', Int16), - ('replica_assignment', Array( - ('partition_id', Int32), - ('replicas', Array(Int32)))), - ('configs', Array( - ('config_key', String('utf-8')), - ('config_value', String('utf-8')))))), - ('timeout', Int32) - ) - - -class CreateTopicsRequest_v1(Request): - API_KEY = 19 - API_VERSION = 1 - RESPONSE_TYPE = CreateTopicsResponse_v1 - SCHEMA = Schema( - ('create_topic_requests', Array( - ('topic', String('utf-8')), - ('num_partitions', Int32), - ('replication_factor', Int16), - ('replica_assignment', Array( - ('partition_id', Int32), - ('replicas', Array(Int32)))), - ('configs', Array( - ('config_key', String('utf-8')), - ('config_value', String('utf-8')))))), - ('timeout', Int32), - ('validate_only', Boolean) - ) - - -class CreateTopicsRequest_v2(Request): - API_KEY = 19 - API_VERSION = 2 - RESPONSE_TYPE = CreateTopicsResponse_v2 - SCHEMA = CreateTopicsRequest_v1.SCHEMA - - -class CreateTopicsRequest_v3(Request): - API_KEY = 19 - API_VERSION = 3 - RESPONSE_TYPE = CreateTopicsResponse_v3 - SCHEMA = CreateTopicsRequest_v1.SCHEMA - - -CreateTopicsRequest = [ - CreateTopicsRequest_v0, CreateTopicsRequest_v1, - CreateTopicsRequest_v2, CreateTopicsRequest_v3, -] -CreateTopicsResponse = [ - CreateTopicsResponse_v0, CreateTopicsResponse_v1, - CreateTopicsResponse_v2, CreateTopicsResponse_v3, -] - - -class DeleteTopicsResponse_v0(Response): - API_KEY = 20 - API_VERSION = 0 - SCHEMA = Schema( - ('topic_error_codes', Array( - ('topic', String('utf-8')), - ('error_code', Int16))) - ) - - -class DeleteTopicsResponse_v1(Response): - API_KEY = 20 - API_VERSION = 1 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('topic_error_codes', Array( - ('topic', String('utf-8')), - ('error_code', Int16))) - ) - - -class DeleteTopicsResponse_v2(Response): - API_KEY = 20 - API_VERSION = 2 - SCHEMA = DeleteTopicsResponse_v1.SCHEMA - - -class DeleteTopicsResponse_v3(Response): - API_KEY = 20 - API_VERSION = 3 - SCHEMA = DeleteTopicsResponse_v1.SCHEMA - - -class DeleteTopicsRequest_v0(Request): - API_KEY = 20 - API_VERSION = 0 - RESPONSE_TYPE = DeleteTopicsResponse_v0 - SCHEMA = Schema( - ('topics', Array(String('utf-8'))), - ('timeout', Int32) - ) - - -class DeleteTopicsRequest_v1(Request): - API_KEY = 20 - API_VERSION = 1 - RESPONSE_TYPE = DeleteTopicsResponse_v1 - SCHEMA = DeleteTopicsRequest_v0.SCHEMA - - -class DeleteTopicsRequest_v2(Request): - API_KEY = 20 - API_VERSION = 2 - RESPONSE_TYPE = DeleteTopicsResponse_v2 - SCHEMA = DeleteTopicsRequest_v0.SCHEMA - - -class DeleteTopicsRequest_v3(Request): - API_KEY = 20 - API_VERSION = 3 - RESPONSE_TYPE = DeleteTopicsResponse_v3 - SCHEMA = DeleteTopicsRequest_v0.SCHEMA - - -DeleteTopicsRequest = [ - DeleteTopicsRequest_v0, DeleteTopicsRequest_v1, - DeleteTopicsRequest_v2, DeleteTopicsRequest_v3, -] -DeleteTopicsResponse = [ - DeleteTopicsResponse_v0, DeleteTopicsResponse_v1, - DeleteTopicsResponse_v2, DeleteTopicsResponse_v3, -] - - -class ListGroupsResponse_v0(Response): - API_KEY = 16 - API_VERSION = 0 - SCHEMA = Schema( - ('error_code', Int16), - ('groups', Array( - ('group', String('utf-8')), - ('protocol_type', String('utf-8')))) - ) - - -class ListGroupsResponse_v1(Response): - API_KEY = 16 - API_VERSION = 1 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('error_code', Int16), - ('groups', Array( - ('group', String('utf-8')), - ('protocol_type', String('utf-8')))) - ) - -class ListGroupsResponse_v2(Response): - API_KEY = 16 - API_VERSION = 2 - SCHEMA = ListGroupsResponse_v1.SCHEMA - - -class ListGroupsRequest_v0(Request): - API_KEY = 16 - API_VERSION = 0 - RESPONSE_TYPE = ListGroupsResponse_v0 - SCHEMA = Schema() - - -class ListGroupsRequest_v1(Request): - API_KEY = 16 - API_VERSION = 1 - RESPONSE_TYPE = ListGroupsResponse_v1 - SCHEMA = ListGroupsRequest_v0.SCHEMA - -class ListGroupsRequest_v2(Request): - API_KEY = 16 - API_VERSION = 1 - RESPONSE_TYPE = ListGroupsResponse_v2 - SCHEMA = ListGroupsRequest_v0.SCHEMA - - -ListGroupsRequest = [ - ListGroupsRequest_v0, ListGroupsRequest_v1, - ListGroupsRequest_v2, -] -ListGroupsResponse = [ - ListGroupsResponse_v0, ListGroupsResponse_v1, - ListGroupsResponse_v2, -] - - -class DescribeGroupsResponse_v0(Response): - API_KEY = 15 - API_VERSION = 0 - SCHEMA = Schema( - ('groups', Array( - ('error_code', Int16), - ('group', String('utf-8')), - ('state', String('utf-8')), - ('protocol_type', String('utf-8')), - ('protocol', String('utf-8')), - ('members', Array( - ('member_id', String('utf-8')), - ('client_id', String('utf-8')), - ('client_host', String('utf-8')), - ('member_metadata', Bytes), - ('member_assignment', Bytes))))) - ) - - -class DescribeGroupsResponse_v1(Response): - API_KEY = 15 - API_VERSION = 1 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('groups', Array( - ('error_code', Int16), - ('group', String('utf-8')), - ('state', String('utf-8')), - ('protocol_type', String('utf-8')), - ('protocol', String('utf-8')), - ('members', Array( - ('member_id', String('utf-8')), - ('client_id', String('utf-8')), - ('client_host', String('utf-8')), - ('member_metadata', Bytes), - ('member_assignment', Bytes))))) - ) - - -class DescribeGroupsResponse_v2(Response): - API_KEY = 15 - API_VERSION = 2 - SCHEMA = DescribeGroupsResponse_v1.SCHEMA - - -class DescribeGroupsResponse_v3(Response): - API_KEY = 15 - API_VERSION = 3 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('groups', Array( - ('error_code', Int16), - ('group', String('utf-8')), - ('state', String('utf-8')), - ('protocol_type', String('utf-8')), - ('protocol', String('utf-8')), - ('members', Array( - ('member_id', String('utf-8')), - ('client_id', String('utf-8')), - ('client_host', String('utf-8')), - ('member_metadata', Bytes), - ('member_assignment', Bytes)))), - ('authorized_operations', Int32)) - ) - - -class DescribeGroupsRequest_v0(Request): - API_KEY = 15 - API_VERSION = 0 - RESPONSE_TYPE = DescribeGroupsResponse_v0 - SCHEMA = Schema( - ('groups', Array(String('utf-8'))) - ) - - -class DescribeGroupsRequest_v1(Request): - API_KEY = 15 - API_VERSION = 1 - RESPONSE_TYPE = DescribeGroupsResponse_v1 - SCHEMA = DescribeGroupsRequest_v0.SCHEMA - - -class DescribeGroupsRequest_v2(Request): - API_KEY = 15 - API_VERSION = 2 - RESPONSE_TYPE = DescribeGroupsResponse_v2 - SCHEMA = DescribeGroupsRequest_v0.SCHEMA - - -class DescribeGroupsRequest_v3(Request): - API_KEY = 15 - API_VERSION = 3 - RESPONSE_TYPE = DescribeGroupsResponse_v2 - SCHEMA = Schema( - ('groups', Array(String('utf-8'))), - ('include_authorized_operations', Boolean) - ) - - -DescribeGroupsRequest = [ - DescribeGroupsRequest_v0, DescribeGroupsRequest_v1, - DescribeGroupsRequest_v2, DescribeGroupsRequest_v3, -] -DescribeGroupsResponse = [ - DescribeGroupsResponse_v0, DescribeGroupsResponse_v1, - DescribeGroupsResponse_v2, DescribeGroupsResponse_v3, -] - - -class SaslHandShakeResponse_v0(Response): - API_KEY = 17 - API_VERSION = 0 - SCHEMA = Schema( - ('error_code', Int16), - ('enabled_mechanisms', Array(String('utf-8'))) - ) - - -class SaslHandShakeResponse_v1(Response): - API_KEY = 17 - API_VERSION = 1 - SCHEMA = SaslHandShakeResponse_v0.SCHEMA - - -class SaslHandShakeRequest_v0(Request): - API_KEY = 17 - API_VERSION = 0 - RESPONSE_TYPE = SaslHandShakeResponse_v0 - SCHEMA = Schema( - ('mechanism', String('utf-8')) - ) - - -class SaslHandShakeRequest_v1(Request): - API_KEY = 17 - API_VERSION = 1 - RESPONSE_TYPE = SaslHandShakeResponse_v1 - SCHEMA = SaslHandShakeRequest_v0.SCHEMA - - -SaslHandShakeRequest = [SaslHandShakeRequest_v0, SaslHandShakeRequest_v1] -SaslHandShakeResponse = [SaslHandShakeResponse_v0, SaslHandShakeResponse_v1] - - -class DescribeAclsResponse_v0(Response): - API_KEY = 29 - API_VERSION = 0 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('error_code', Int16), - ('error_message', String('utf-8')), - ('resources', Array( - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('acls', Array( - ('principal', String('utf-8')), - ('host', String('utf-8')), - ('operation', Int8), - ('permission_type', Int8))))) - ) - - -class DescribeAclsResponse_v1(Response): - API_KEY = 29 - API_VERSION = 1 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('error_code', Int16), - ('error_message', String('utf-8')), - ('resources', Array( - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('resource_pattern_type', Int8), - ('acls', Array( - ('principal', String('utf-8')), - ('host', String('utf-8')), - ('operation', Int8), - ('permission_type', Int8))))) - ) - - -class DescribeAclsResponse_v2(Response): - API_KEY = 29 - API_VERSION = 2 - SCHEMA = DescribeAclsResponse_v1.SCHEMA - - -class DescribeAclsRequest_v0(Request): - API_KEY = 29 - API_VERSION = 0 - RESPONSE_TYPE = DescribeAclsResponse_v0 - SCHEMA = Schema( - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('principal', String('utf-8')), - ('host', String('utf-8')), - ('operation', Int8), - ('permission_type', Int8) - ) - - -class DescribeAclsRequest_v1(Request): - API_KEY = 29 - API_VERSION = 1 - RESPONSE_TYPE = DescribeAclsResponse_v1 - SCHEMA = Schema( - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('resource_pattern_type_filter', Int8), - ('principal', String('utf-8')), - ('host', String('utf-8')), - ('operation', Int8), - ('permission_type', Int8) - ) - - -class DescribeAclsRequest_v2(Request): - """ - Enable flexible version - """ - API_KEY = 29 - API_VERSION = 2 - RESPONSE_TYPE = DescribeAclsResponse_v2 - SCHEMA = DescribeAclsRequest_v1.SCHEMA - - -DescribeAclsRequest = [DescribeAclsRequest_v0, DescribeAclsRequest_v1] -DescribeAclsResponse = [DescribeAclsResponse_v0, DescribeAclsResponse_v1] - -class CreateAclsResponse_v0(Response): - API_KEY = 30 - API_VERSION = 0 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('creation_responses', Array( - ('error_code', Int16), - ('error_message', String('utf-8')))) - ) - -class CreateAclsResponse_v1(Response): - API_KEY = 30 - API_VERSION = 1 - SCHEMA = CreateAclsResponse_v0.SCHEMA - -class CreateAclsRequest_v0(Request): - API_KEY = 30 - API_VERSION = 0 - RESPONSE_TYPE = CreateAclsResponse_v0 - SCHEMA = Schema( - ('creations', Array( - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('principal', String('utf-8')), - ('host', String('utf-8')), - ('operation', Int8), - ('permission_type', Int8))) - ) - -class CreateAclsRequest_v1(Request): - API_KEY = 30 - API_VERSION = 1 - RESPONSE_TYPE = CreateAclsResponse_v1 - SCHEMA = Schema( - ('creations', Array( - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('resource_pattern_type', Int8), - ('principal', String('utf-8')), - ('host', String('utf-8')), - ('operation', Int8), - ('permission_type', Int8))) - ) - -CreateAclsRequest = [CreateAclsRequest_v0, CreateAclsRequest_v1] -CreateAclsResponse = [CreateAclsResponse_v0, CreateAclsResponse_v1] - -class DeleteAclsResponse_v0(Response): - API_KEY = 31 - API_VERSION = 0 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('filter_responses', Array( - ('error_code', Int16), - ('error_message', String('utf-8')), - ('matching_acls', Array( - ('error_code', Int16), - ('error_message', String('utf-8')), - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('principal', String('utf-8')), - ('host', String('utf-8')), - ('operation', Int8), - ('permission_type', Int8))))) - ) - -class DeleteAclsResponse_v1(Response): - API_KEY = 31 - API_VERSION = 1 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('filter_responses', Array( - ('error_code', Int16), - ('error_message', String('utf-8')), - ('matching_acls', Array( - ('error_code', Int16), - ('error_message', String('utf-8')), - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('resource_pattern_type', Int8), - ('principal', String('utf-8')), - ('host', String('utf-8')), - ('operation', Int8), - ('permission_type', Int8))))) - ) - -class DeleteAclsRequest_v0(Request): - API_KEY = 31 - API_VERSION = 0 - RESPONSE_TYPE = DeleteAclsResponse_v0 - SCHEMA = Schema( - ('filters', Array( - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('principal', String('utf-8')), - ('host', String('utf-8')), - ('operation', Int8), - ('permission_type', Int8))) - ) - -class DeleteAclsRequest_v1(Request): - API_KEY = 31 - API_VERSION = 1 - RESPONSE_TYPE = DeleteAclsResponse_v1 - SCHEMA = Schema( - ('filters', Array( - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('resource_pattern_type_filter', Int8), - ('principal', String('utf-8')), - ('host', String('utf-8')), - ('operation', Int8), - ('permission_type', Int8))) - ) - -DeleteAclsRequest = [DeleteAclsRequest_v0, DeleteAclsRequest_v1] -DeleteAclsResponse = [DeleteAclsResponse_v0, DeleteAclsResponse_v1] - -class AlterConfigsResponse_v0(Response): - API_KEY = 33 - API_VERSION = 0 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('resources', Array( - ('error_code', Int16), - ('error_message', String('utf-8')), - ('resource_type', Int8), - ('resource_name', String('utf-8')))) - ) - - -class AlterConfigsResponse_v1(Response): - API_KEY = 33 - API_VERSION = 1 - SCHEMA = AlterConfigsResponse_v0.SCHEMA - - -class AlterConfigsRequest_v0(Request): - API_KEY = 33 - API_VERSION = 0 - RESPONSE_TYPE = AlterConfigsResponse_v0 - SCHEMA = Schema( - ('resources', Array( - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('config_entries', Array( - ('config_name', String('utf-8')), - ('config_value', String('utf-8')))))), - ('validate_only', Boolean) - ) - -class AlterConfigsRequest_v1(Request): - API_KEY = 33 - API_VERSION = 1 - RESPONSE_TYPE = AlterConfigsResponse_v1 - SCHEMA = AlterConfigsRequest_v0.SCHEMA - -AlterConfigsRequest = [AlterConfigsRequest_v0, AlterConfigsRequest_v1] -AlterConfigsResponse = [AlterConfigsResponse_v0, AlterConfigsRequest_v1] - - -class DescribeConfigsResponse_v0(Response): - API_KEY = 32 - API_VERSION = 0 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('resources', Array( - ('error_code', Int16), - ('error_message', String('utf-8')), - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('config_entries', Array( - ('config_names', String('utf-8')), - ('config_value', String('utf-8')), - ('read_only', Boolean), - ('is_default', Boolean), - ('is_sensitive', Boolean))))) - ) - -class DescribeConfigsResponse_v1(Response): - API_KEY = 32 - API_VERSION = 1 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('resources', Array( - ('error_code', Int16), - ('error_message', String('utf-8')), - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('config_entries', Array( - ('config_names', String('utf-8')), - ('config_value', String('utf-8')), - ('read_only', Boolean), - ('is_default', Boolean), - ('is_sensitive', Boolean), - ('config_synonyms', Array( - ('config_name', String('utf-8')), - ('config_value', String('utf-8')), - ('config_source', Int8))))))) - ) - -class DescribeConfigsResponse_v2(Response): - API_KEY = 32 - API_VERSION = 2 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('resources', Array( - ('error_code', Int16), - ('error_message', String('utf-8')), - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('config_entries', Array( - ('config_names', String('utf-8')), - ('config_value', String('utf-8')), - ('read_only', Boolean), - ('config_source', Int8), - ('is_sensitive', Boolean), - ('config_synonyms', Array( - ('config_name', String('utf-8')), - ('config_value', String('utf-8')), - ('config_source', Int8))))))) - ) - -class DescribeConfigsRequest_v0(Request): - API_KEY = 32 - API_VERSION = 0 - RESPONSE_TYPE = DescribeConfigsResponse_v0 - SCHEMA = Schema( - ('resources', Array( - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('config_names', Array(String('utf-8'))))) - ) - -class DescribeConfigsRequest_v1(Request): - API_KEY = 32 - API_VERSION = 1 - RESPONSE_TYPE = DescribeConfigsResponse_v1 - SCHEMA = Schema( - ('resources', Array( - ('resource_type', Int8), - ('resource_name', String('utf-8')), - ('config_names', Array(String('utf-8'))))), - ('include_synonyms', Boolean) - ) - - -class DescribeConfigsRequest_v2(Request): - API_KEY = 32 - API_VERSION = 2 - RESPONSE_TYPE = DescribeConfigsResponse_v2 - SCHEMA = DescribeConfigsRequest_v1.SCHEMA - - -DescribeConfigsRequest = [ - DescribeConfigsRequest_v0, DescribeConfigsRequest_v1, - DescribeConfigsRequest_v2, -] -DescribeConfigsResponse = [ - DescribeConfigsResponse_v0, DescribeConfigsResponse_v1, - DescribeConfigsResponse_v2, -] - - -class SaslAuthenticateResponse_v0(Response): - API_KEY = 36 - API_VERSION = 0 - SCHEMA = Schema( - ('error_code', Int16), - ('error_message', String('utf-8')), - ('sasl_auth_bytes', Bytes) - ) - - -class SaslAuthenticateResponse_v1(Response): - API_KEY = 36 - API_VERSION = 1 - SCHEMA = Schema( - ('error_code', Int16), - ('error_message', String('utf-8')), - ('sasl_auth_bytes', Bytes), - ('session_lifetime_ms', Int64) - ) - - -class SaslAuthenticateRequest_v0(Request): - API_KEY = 36 - API_VERSION = 0 - RESPONSE_TYPE = SaslAuthenticateResponse_v0 - SCHEMA = Schema( - ('sasl_auth_bytes', Bytes) - ) - - -class SaslAuthenticateRequest_v1(Request): - API_KEY = 36 - API_VERSION = 1 - RESPONSE_TYPE = SaslAuthenticateResponse_v1 - SCHEMA = SaslAuthenticateRequest_v0.SCHEMA - - -SaslAuthenticateRequest = [ - SaslAuthenticateRequest_v0, SaslAuthenticateRequest_v1, -] -SaslAuthenticateResponse = [ - SaslAuthenticateResponse_v0, SaslAuthenticateResponse_v1, -] - - -class CreatePartitionsResponse_v0(Response): - API_KEY = 37 - API_VERSION = 0 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('topic_errors', Array( - ('topic', String('utf-8')), - ('error_code', Int16), - ('error_message', String('utf-8')))) - ) - - -class CreatePartitionsResponse_v1(Response): - API_KEY = 37 - API_VERSION = 1 - SCHEMA = CreatePartitionsResponse_v0.SCHEMA - - -class CreatePartitionsRequest_v0(Request): - API_KEY = 37 - API_VERSION = 0 - RESPONSE_TYPE = CreatePartitionsResponse_v0 - SCHEMA = Schema( - ('topic_partitions', Array( - ('topic', String('utf-8')), - ('new_partitions', Schema( - ('count', Int32), - ('assignment', Array(Array(Int32))))))), - ('timeout', Int32), - ('validate_only', Boolean) - ) - - -class CreatePartitionsRequest_v1(Request): - API_KEY = 37 - API_VERSION = 1 - SCHEMA = CreatePartitionsRequest_v0.SCHEMA - RESPONSE_TYPE = CreatePartitionsResponse_v1 - - -CreatePartitionsRequest = [ - CreatePartitionsRequest_v0, CreatePartitionsRequest_v1, -] -CreatePartitionsResponse = [ - CreatePartitionsResponse_v0, CreatePartitionsResponse_v1, -] - - -class DeleteGroupsResponse_v0(Response): - API_KEY = 42 - API_VERSION = 0 - SCHEMA = Schema( - ("throttle_time_ms", Int32), - ("results", Array( - ("group_id", String("utf-8")), - ("error_code", Int16))) - ) - - -class DeleteGroupsResponse_v1(Response): - API_KEY = 42 - API_VERSION = 1 - SCHEMA = DeleteGroupsResponse_v0.SCHEMA - - -class DeleteGroupsRequest_v0(Request): - API_KEY = 42 - API_VERSION = 0 - RESPONSE_TYPE = DeleteGroupsResponse_v0 - SCHEMA = Schema( - ("groups_names", Array(String("utf-8"))) - ) - - -class DeleteGroupsRequest_v1(Request): - API_KEY = 42 - API_VERSION = 1 - RESPONSE_TYPE = DeleteGroupsResponse_v1 - SCHEMA = DeleteGroupsRequest_v0.SCHEMA - - -DeleteGroupsRequest = [ - DeleteGroupsRequest_v0, DeleteGroupsRequest_v1 -] - -DeleteGroupsResponse = [ - DeleteGroupsResponse_v0, DeleteGroupsResponse_v1 -] - - -class DescribeClientQuotasResponse_v0(Request): - API_KEY = 48 - API_VERSION = 0 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('error_code', Int16), - ('error_message', String('utf-8')), - ('entries', Array( - ('entity', Array( - ('entity_type', String('utf-8')), - ('entity_name', String('utf-8')))), - ('values', Array( - ('name', String('utf-8')), - ('value', Float64))))), - ) - - -class DescribeClientQuotasRequest_v0(Request): - API_KEY = 48 - API_VERSION = 0 - RESPONSE_TYPE = DescribeClientQuotasResponse_v0 - SCHEMA = Schema( - ('components', Array( - ('entity_type', String('utf-8')), - ('match_type', Int8), - ('match', String('utf-8')), - )), - ('strict', Boolean) - ) - - -DescribeClientQuotasRequest = [ - DescribeClientQuotasRequest_v0, -] - -DescribeClientQuotasResponse = [ - DescribeClientQuotasResponse_v0, -] - - -class AlterPartitionReassignmentsResponse_v0(Response): - API_KEY = 45 - API_VERSION = 0 - SCHEMA = Schema( - ("throttle_time_ms", Int32), - ("error_code", Int16), - ("error_message", CompactString("utf-8")), - ("responses", CompactArray( - ("name", CompactString("utf-8")), - ("partitions", CompactArray( - ("partition_index", Int32), - ("error_code", Int16), - ("error_message", CompactString("utf-8")), - ("tags", TaggedFields) - )), - ("tags", TaggedFields) - )), - ("tags", TaggedFields) - ) - - -class AlterPartitionReassignmentsRequest_v0(Request): - FLEXIBLE_VERSION = True - API_KEY = 45 - API_VERSION = 0 - RESPONSE_TYPE = AlterPartitionReassignmentsResponse_v0 - SCHEMA = Schema( - ("timeout_ms", Int32), - ("topics", CompactArray( - ("name", CompactString("utf-8")), - ("partitions", CompactArray( - ("partition_index", Int32), - ("replicas", CompactArray(Int32)), - ("tags", TaggedFields) - )), - ("tags", TaggedFields) - )), - ("tags", TaggedFields) - ) - - -AlterPartitionReassignmentsRequest = [AlterPartitionReassignmentsRequest_v0] - -AlterPartitionReassignmentsResponse = [AlterPartitionReassignmentsResponse_v0] - - -class ListPartitionReassignmentsResponse_v0(Response): - API_KEY = 46 - API_VERSION = 0 - SCHEMA = Schema( - ("throttle_time_ms", Int32), - ("error_code", Int16), - ("error_message", CompactString("utf-8")), - ("topics", CompactArray( - ("name", CompactString("utf-8")), - ("partitions", CompactArray( - ("partition_index", Int32), - ("replicas", CompactArray(Int32)), - ("adding_replicas", CompactArray(Int32)), - ("removing_replicas", CompactArray(Int32)), - ("tags", TaggedFields) - )), - ("tags", TaggedFields) - )), - ("tags", TaggedFields) - ) - - -class ListPartitionReassignmentsRequest_v0(Request): - FLEXIBLE_VERSION = True - API_KEY = 46 - API_VERSION = 0 - RESPONSE_TYPE = ListPartitionReassignmentsResponse_v0 - SCHEMA = Schema( - ("timeout_ms", Int32), - ("topics", CompactArray( - ("name", CompactString("utf-8")), - ("partition_index", CompactArray(Int32)), - ("tags", TaggedFields) - )), - ("tags", TaggedFields) - ) - - -ListPartitionReassignmentsRequest = [ListPartitionReassignmentsRequest_v0] - -ListPartitionReassignmentsResponse = [ListPartitionReassignmentsResponse_v0] diff --git a/kafka/protocol/commit.py b/kafka/protocol/commit.py deleted file mode 100644 index 31fc2370..00000000 --- a/kafka/protocol/commit.py +++ /dev/null @@ -1,255 +0,0 @@ -from __future__ import absolute_import - -from kafka.protocol.api import Request, Response -from kafka.protocol.types import Array, Int8, Int16, Int32, Int64, Schema, String - - -class OffsetCommitResponse_v0(Response): - API_KEY = 8 - API_VERSION = 0 - SCHEMA = Schema( - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16))))) - ) - - -class OffsetCommitResponse_v1(Response): - API_KEY = 8 - API_VERSION = 1 - SCHEMA = OffsetCommitResponse_v0.SCHEMA - - -class OffsetCommitResponse_v2(Response): - API_KEY = 8 - API_VERSION = 2 - SCHEMA = OffsetCommitResponse_v1.SCHEMA - - -class OffsetCommitResponse_v3(Response): - API_KEY = 8 - API_VERSION = 3 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16))))) - ) - - -class OffsetCommitRequest_v0(Request): - API_KEY = 8 - API_VERSION = 0 # Zookeeper-backed storage - RESPONSE_TYPE = OffsetCommitResponse_v0 - SCHEMA = Schema( - ('consumer_group', String('utf-8')), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('offset', Int64), - ('metadata', String('utf-8')))))) - ) - - -class OffsetCommitRequest_v1(Request): - API_KEY = 8 - API_VERSION = 1 # Kafka-backed storage - RESPONSE_TYPE = OffsetCommitResponse_v1 - SCHEMA = Schema( - ('consumer_group', String('utf-8')), - ('consumer_group_generation_id', Int32), - ('consumer_id', String('utf-8')), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('offset', Int64), - ('timestamp', Int64), - ('metadata', String('utf-8')))))) - ) - - -class OffsetCommitRequest_v2(Request): - API_KEY = 8 - API_VERSION = 2 # added retention_time, dropped timestamp - RESPONSE_TYPE = OffsetCommitResponse_v2 - SCHEMA = Schema( - ('consumer_group', String('utf-8')), - ('consumer_group_generation_id', Int32), - ('consumer_id', String('utf-8')), - ('retention_time', Int64), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('offset', Int64), - ('metadata', String('utf-8')))))) - ) - DEFAULT_GENERATION_ID = -1 - DEFAULT_RETENTION_TIME = -1 - - -class OffsetCommitRequest_v3(Request): - API_KEY = 8 - API_VERSION = 3 - RESPONSE_TYPE = OffsetCommitResponse_v3 - SCHEMA = OffsetCommitRequest_v2.SCHEMA - - -OffsetCommitRequest = [ - OffsetCommitRequest_v0, OffsetCommitRequest_v1, - OffsetCommitRequest_v2, OffsetCommitRequest_v3 -] -OffsetCommitResponse = [ - OffsetCommitResponse_v0, OffsetCommitResponse_v1, - OffsetCommitResponse_v2, OffsetCommitResponse_v3 -] - - -class OffsetFetchResponse_v0(Response): - API_KEY = 9 - API_VERSION = 0 - SCHEMA = Schema( - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('offset', Int64), - ('metadata', String('utf-8')), - ('error_code', Int16))))) - ) - - -class OffsetFetchResponse_v1(Response): - API_KEY = 9 - API_VERSION = 1 - SCHEMA = OffsetFetchResponse_v0.SCHEMA - - -class OffsetFetchResponse_v2(Response): - # Added in KIP-88 - API_KEY = 9 - API_VERSION = 2 - SCHEMA = Schema( - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('offset', Int64), - ('metadata', String('utf-8')), - ('error_code', Int16))))), - ('error_code', Int16) - ) - - -class OffsetFetchResponse_v3(Response): - API_KEY = 9 - API_VERSION = 3 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('offset', Int64), - ('metadata', String('utf-8')), - ('error_code', Int16))))), - ('error_code', Int16) - ) - - -class OffsetFetchRequest_v0(Request): - API_KEY = 9 - API_VERSION = 0 # zookeeper-backed storage - RESPONSE_TYPE = OffsetFetchResponse_v0 - SCHEMA = Schema( - ('consumer_group', String('utf-8')), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array(Int32)))) - ) - - -class OffsetFetchRequest_v1(Request): - API_KEY = 9 - API_VERSION = 1 # kafka-backed storage - RESPONSE_TYPE = OffsetFetchResponse_v1 - SCHEMA = OffsetFetchRequest_v0.SCHEMA - - -class OffsetFetchRequest_v2(Request): - # KIP-88: Allows passing null topics to return offsets for all partitions - # that the consumer group has a stored offset for, even if no consumer in - # the group is currently consuming that partition. - API_KEY = 9 - API_VERSION = 2 - RESPONSE_TYPE = OffsetFetchResponse_v2 - SCHEMA = OffsetFetchRequest_v1.SCHEMA - - -class OffsetFetchRequest_v3(Request): - API_KEY = 9 - API_VERSION = 3 - RESPONSE_TYPE = OffsetFetchResponse_v3 - SCHEMA = OffsetFetchRequest_v2.SCHEMA - - -OffsetFetchRequest = [ - OffsetFetchRequest_v0, OffsetFetchRequest_v1, - OffsetFetchRequest_v2, OffsetFetchRequest_v3, -] -OffsetFetchResponse = [ - OffsetFetchResponse_v0, OffsetFetchResponse_v1, - OffsetFetchResponse_v2, OffsetFetchResponse_v3, -] - - -class GroupCoordinatorResponse_v0(Response): - API_KEY = 10 - API_VERSION = 0 - SCHEMA = Schema( - ('error_code', Int16), - ('coordinator_id', Int32), - ('host', String('utf-8')), - ('port', Int32) - ) - - -class GroupCoordinatorResponse_v1(Response): - API_KEY = 10 - API_VERSION = 1 - SCHEMA = Schema( - ('error_code', Int16), - ('error_message', String('utf-8')), - ('coordinator_id', Int32), - ('host', String('utf-8')), - ('port', Int32) - ) - - -class GroupCoordinatorRequest_v0(Request): - API_KEY = 10 - API_VERSION = 0 - RESPONSE_TYPE = GroupCoordinatorResponse_v0 - SCHEMA = Schema( - ('consumer_group', String('utf-8')) - ) - - -class GroupCoordinatorRequest_v1(Request): - API_KEY = 10 - API_VERSION = 1 - RESPONSE_TYPE = GroupCoordinatorResponse_v1 - SCHEMA = Schema( - ('coordinator_key', String('utf-8')), - ('coordinator_type', Int8) - ) - - -GroupCoordinatorRequest = [GroupCoordinatorRequest_v0, GroupCoordinatorRequest_v1] -GroupCoordinatorResponse = [GroupCoordinatorResponse_v0, GroupCoordinatorResponse_v1] diff --git a/kafka/protocol/fetch.py b/kafka/protocol/fetch.py deleted file mode 100644 index f367848c..00000000 --- a/kafka/protocol/fetch.py +++ /dev/null @@ -1,386 +0,0 @@ -from __future__ import absolute_import - -from kafka.protocol.api import Request, Response -from kafka.protocol.types import Array, Int8, Int16, Int32, Int64, Schema, String, Bytes - - -class FetchResponse_v0(Response): - API_KEY = 1 - API_VERSION = 0 - SCHEMA = Schema( - ('topics', Array( - ('topics', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('highwater_offset', Int64), - ('message_set', Bytes))))) - ) - - -class FetchResponse_v1(Response): - API_KEY = 1 - API_VERSION = 1 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('topics', Array( - ('topics', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('highwater_offset', Int64), - ('message_set', Bytes))))) - ) - - -class FetchResponse_v2(Response): - API_KEY = 1 - API_VERSION = 2 - SCHEMA = FetchResponse_v1.SCHEMA # message format changed internally - - -class FetchResponse_v3(Response): - API_KEY = 1 - API_VERSION = 3 - SCHEMA = FetchResponse_v2.SCHEMA - - -class FetchResponse_v4(Response): - API_KEY = 1 - API_VERSION = 4 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('topics', Array( - ('topics', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('highwater_offset', Int64), - ('last_stable_offset', Int64), - ('aborted_transactions', Array( - ('producer_id', Int64), - ('first_offset', Int64))), - ('message_set', Bytes))))) - ) - - -class FetchResponse_v5(Response): - API_KEY = 1 - API_VERSION = 5 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('topics', Array( - ('topics', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('highwater_offset', Int64), - ('last_stable_offset', Int64), - ('log_start_offset', Int64), - ('aborted_transactions', Array( - ('producer_id', Int64), - ('first_offset', Int64))), - ('message_set', Bytes))))) - ) - - -class FetchResponse_v6(Response): - """ - Same as FetchResponse_v5. The version number is bumped up to indicate that the client supports KafkaStorageException. - The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 5 - """ - API_KEY = 1 - API_VERSION = 6 - SCHEMA = FetchResponse_v5.SCHEMA - - -class FetchResponse_v7(Response): - """ - Add error_code and session_id to response - """ - API_KEY = 1 - API_VERSION = 7 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('error_code', Int16), - ('session_id', Int32), - ('topics', Array( - ('topics', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('highwater_offset', Int64), - ('last_stable_offset', Int64), - ('log_start_offset', Int64), - ('aborted_transactions', Array( - ('producer_id', Int64), - ('first_offset', Int64))), - ('message_set', Bytes))))) - ) - - -class FetchResponse_v8(Response): - API_KEY = 1 - API_VERSION = 8 - SCHEMA = FetchResponse_v7.SCHEMA - - -class FetchResponse_v9(Response): - API_KEY = 1 - API_VERSION = 9 - SCHEMA = FetchResponse_v7.SCHEMA - - -class FetchResponse_v10(Response): - API_KEY = 1 - API_VERSION = 10 - SCHEMA = FetchResponse_v7.SCHEMA - - -class FetchResponse_v11(Response): - API_KEY = 1 - API_VERSION = 11 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('error_code', Int16), - ('session_id', Int32), - ('topics', Array( - ('topics', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('highwater_offset', Int64), - ('last_stable_offset', Int64), - ('log_start_offset', Int64), - ('aborted_transactions', Array( - ('producer_id', Int64), - ('first_offset', Int64))), - ('preferred_read_replica', Int32), - ('message_set', Bytes))))) - ) - - -class FetchRequest_v0(Request): - API_KEY = 1 - API_VERSION = 0 - RESPONSE_TYPE = FetchResponse_v0 - SCHEMA = Schema( - ('replica_id', Int32), - ('max_wait_time', Int32), - ('min_bytes', Int32), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('offset', Int64), - ('max_bytes', Int32))))) - ) - - -class FetchRequest_v1(Request): - API_KEY = 1 - API_VERSION = 1 - RESPONSE_TYPE = FetchResponse_v1 - SCHEMA = FetchRequest_v0.SCHEMA - - -class FetchRequest_v2(Request): - API_KEY = 1 - API_VERSION = 2 - RESPONSE_TYPE = FetchResponse_v2 - SCHEMA = FetchRequest_v1.SCHEMA - - -class FetchRequest_v3(Request): - API_KEY = 1 - API_VERSION = 3 - RESPONSE_TYPE = FetchResponse_v3 - SCHEMA = Schema( - ('replica_id', Int32), - ('max_wait_time', Int32), - ('min_bytes', Int32), - ('max_bytes', Int32), # This new field is only difference from FR_v2 - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('offset', Int64), - ('max_bytes', Int32))))) - ) - - -class FetchRequest_v4(Request): - # Adds isolation_level field - API_KEY = 1 - API_VERSION = 4 - RESPONSE_TYPE = FetchResponse_v4 - SCHEMA = Schema( - ('replica_id', Int32), - ('max_wait_time', Int32), - ('min_bytes', Int32), - ('max_bytes', Int32), - ('isolation_level', Int8), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('offset', Int64), - ('max_bytes', Int32))))) - ) - - -class FetchRequest_v5(Request): - # This may only be used in broker-broker api calls - API_KEY = 1 - API_VERSION = 5 - RESPONSE_TYPE = FetchResponse_v5 - SCHEMA = Schema( - ('replica_id', Int32), - ('max_wait_time', Int32), - ('min_bytes', Int32), - ('max_bytes', Int32), - ('isolation_level', Int8), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('fetch_offset', Int64), - ('log_start_offset', Int64), - ('max_bytes', Int32))))) - ) - - -class FetchRequest_v6(Request): - """ - The body of FETCH_REQUEST_V6 is the same as FETCH_REQUEST_V5. - The version number is bumped up to indicate that the client supports KafkaStorageException. - The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 5 - """ - API_KEY = 1 - API_VERSION = 6 - RESPONSE_TYPE = FetchResponse_v6 - SCHEMA = FetchRequest_v5.SCHEMA - - -class FetchRequest_v7(Request): - """ - Add incremental fetch requests - """ - API_KEY = 1 - API_VERSION = 7 - RESPONSE_TYPE = FetchResponse_v7 - SCHEMA = Schema( - ('replica_id', Int32), - ('max_wait_time', Int32), - ('min_bytes', Int32), - ('max_bytes', Int32), - ('isolation_level', Int8), - ('session_id', Int32), - ('session_epoch', Int32), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('fetch_offset', Int64), - ('log_start_offset', Int64), - ('max_bytes', Int32))))), - ('forgotten_topics_data', Array( - ('topic', String), - ('partitions', Array(Int32)) - )), - ) - - -class FetchRequest_v8(Request): - """ - bump used to indicate that on quota violation brokers send out responses before throttling. - """ - API_KEY = 1 - API_VERSION = 8 - RESPONSE_TYPE = FetchResponse_v8 - SCHEMA = FetchRequest_v7.SCHEMA - - -class FetchRequest_v9(Request): - """ - adds the current leader epoch (see KIP-320) - """ - API_KEY = 1 - API_VERSION = 9 - RESPONSE_TYPE = FetchResponse_v9 - SCHEMA = Schema( - ('replica_id', Int32), - ('max_wait_time', Int32), - ('min_bytes', Int32), - ('max_bytes', Int32), - ('isolation_level', Int8), - ('session_id', Int32), - ('session_epoch', Int32), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('current_leader_epoch', Int32), - ('fetch_offset', Int64), - ('log_start_offset', Int64), - ('max_bytes', Int32))))), - ('forgotten_topics_data', Array( - ('topic', String), - ('partitions', Array(Int32)), - )), - ) - - -class FetchRequest_v10(Request): - """ - bumped up to indicate ZStandard capability. (see KIP-110) - """ - API_KEY = 1 - API_VERSION = 10 - RESPONSE_TYPE = FetchResponse_v10 - SCHEMA = FetchRequest_v9.SCHEMA - - -class FetchRequest_v11(Request): - """ - added rack ID to support read from followers (KIP-392) - """ - API_KEY = 1 - API_VERSION = 11 - RESPONSE_TYPE = FetchResponse_v11 - SCHEMA = Schema( - ('replica_id', Int32), - ('max_wait_time', Int32), - ('min_bytes', Int32), - ('max_bytes', Int32), - ('isolation_level', Int8), - ('session_id', Int32), - ('session_epoch', Int32), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('current_leader_epoch', Int32), - ('fetch_offset', Int64), - ('log_start_offset', Int64), - ('max_bytes', Int32))))), - ('forgotten_topics_data', Array( - ('topic', String), - ('partitions', Array(Int32)) - )), - ('rack_id', String('utf-8')), - ) - - -FetchRequest = [ - FetchRequest_v0, FetchRequest_v1, FetchRequest_v2, - FetchRequest_v3, FetchRequest_v4, FetchRequest_v5, - FetchRequest_v6, FetchRequest_v7, FetchRequest_v8, - FetchRequest_v9, FetchRequest_v10, FetchRequest_v11, -] -FetchResponse = [ - FetchResponse_v0, FetchResponse_v1, FetchResponse_v2, - FetchResponse_v3, FetchResponse_v4, FetchResponse_v5, - FetchResponse_v6, FetchResponse_v7, FetchResponse_v8, - FetchResponse_v9, FetchResponse_v10, FetchResponse_v11, -] diff --git a/kafka/protocol/group.py b/kafka/protocol/group.py deleted file mode 100644 index bcb96553..00000000 --- a/kafka/protocol/group.py +++ /dev/null @@ -1,230 +0,0 @@ -from __future__ import absolute_import - -from kafka.protocol.api import Request, Response -from kafka.protocol.struct import Struct -from kafka.protocol.types import Array, Bytes, Int16, Int32, Schema, String - - -class JoinGroupResponse_v0(Response): - API_KEY = 11 - API_VERSION = 0 - SCHEMA = Schema( - ('error_code', Int16), - ('generation_id', Int32), - ('group_protocol', String('utf-8')), - ('leader_id', String('utf-8')), - ('member_id', String('utf-8')), - ('members', Array( - ('member_id', String('utf-8')), - ('member_metadata', Bytes))) - ) - - -class JoinGroupResponse_v1(Response): - API_KEY = 11 - API_VERSION = 1 - SCHEMA = JoinGroupResponse_v0.SCHEMA - - -class JoinGroupResponse_v2(Response): - API_KEY = 11 - API_VERSION = 2 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('error_code', Int16), - ('generation_id', Int32), - ('group_protocol', String('utf-8')), - ('leader_id', String('utf-8')), - ('member_id', String('utf-8')), - ('members', Array( - ('member_id', String('utf-8')), - ('member_metadata', Bytes))) - ) - - -class JoinGroupRequest_v0(Request): - API_KEY = 11 - API_VERSION = 0 - RESPONSE_TYPE = JoinGroupResponse_v0 - SCHEMA = Schema( - ('group', String('utf-8')), - ('session_timeout', Int32), - ('member_id', String('utf-8')), - ('protocol_type', String('utf-8')), - ('group_protocols', Array( - ('protocol_name', String('utf-8')), - ('protocol_metadata', Bytes))) - ) - UNKNOWN_MEMBER_ID = '' - - -class JoinGroupRequest_v1(Request): - API_KEY = 11 - API_VERSION = 1 - RESPONSE_TYPE = JoinGroupResponse_v1 - SCHEMA = Schema( - ('group', String('utf-8')), - ('session_timeout', Int32), - ('rebalance_timeout', Int32), - ('member_id', String('utf-8')), - ('protocol_type', String('utf-8')), - ('group_protocols', Array( - ('protocol_name', String('utf-8')), - ('protocol_metadata', Bytes))) - ) - UNKNOWN_MEMBER_ID = '' - - -class JoinGroupRequest_v2(Request): - API_KEY = 11 - API_VERSION = 2 - RESPONSE_TYPE = JoinGroupResponse_v2 - SCHEMA = JoinGroupRequest_v1.SCHEMA - UNKNOWN_MEMBER_ID = '' - - -JoinGroupRequest = [ - JoinGroupRequest_v0, JoinGroupRequest_v1, JoinGroupRequest_v2 -] -JoinGroupResponse = [ - JoinGroupResponse_v0, JoinGroupResponse_v1, JoinGroupResponse_v2 -] - - -class ProtocolMetadata(Struct): - SCHEMA = Schema( - ('version', Int16), - ('subscription', Array(String('utf-8'))), # topics list - ('user_data', Bytes) - ) - - -class SyncGroupResponse_v0(Response): - API_KEY = 14 - API_VERSION = 0 - SCHEMA = Schema( - ('error_code', Int16), - ('member_assignment', Bytes) - ) - - -class SyncGroupResponse_v1(Response): - API_KEY = 14 - API_VERSION = 1 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('error_code', Int16), - ('member_assignment', Bytes) - ) - - -class SyncGroupRequest_v0(Request): - API_KEY = 14 - API_VERSION = 0 - RESPONSE_TYPE = SyncGroupResponse_v0 - SCHEMA = Schema( - ('group', String('utf-8')), - ('generation_id', Int32), - ('member_id', String('utf-8')), - ('group_assignment', Array( - ('member_id', String('utf-8')), - ('member_metadata', Bytes))) - ) - - -class SyncGroupRequest_v1(Request): - API_KEY = 14 - API_VERSION = 1 - RESPONSE_TYPE = SyncGroupResponse_v1 - SCHEMA = SyncGroupRequest_v0.SCHEMA - - -SyncGroupRequest = [SyncGroupRequest_v0, SyncGroupRequest_v1] -SyncGroupResponse = [SyncGroupResponse_v0, SyncGroupResponse_v1] - - -class MemberAssignment(Struct): - SCHEMA = Schema( - ('version', Int16), - ('assignment', Array( - ('topic', String('utf-8')), - ('partitions', Array(Int32)))), - ('user_data', Bytes) - ) - - -class HeartbeatResponse_v0(Response): - API_KEY = 12 - API_VERSION = 0 - SCHEMA = Schema( - ('error_code', Int16) - ) - - -class HeartbeatResponse_v1(Response): - API_KEY = 12 - API_VERSION = 1 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('error_code', Int16) - ) - - -class HeartbeatRequest_v0(Request): - API_KEY = 12 - API_VERSION = 0 - RESPONSE_TYPE = HeartbeatResponse_v0 - SCHEMA = Schema( - ('group', String('utf-8')), - ('generation_id', Int32), - ('member_id', String('utf-8')) - ) - - -class HeartbeatRequest_v1(Request): - API_KEY = 12 - API_VERSION = 1 - RESPONSE_TYPE = HeartbeatResponse_v1 - SCHEMA = HeartbeatRequest_v0.SCHEMA - - -HeartbeatRequest = [HeartbeatRequest_v0, HeartbeatRequest_v1] -HeartbeatResponse = [HeartbeatResponse_v0, HeartbeatResponse_v1] - - -class LeaveGroupResponse_v0(Response): - API_KEY = 13 - API_VERSION = 0 - SCHEMA = Schema( - ('error_code', Int16) - ) - - -class LeaveGroupResponse_v1(Response): - API_KEY = 13 - API_VERSION = 1 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('error_code', Int16) - ) - - -class LeaveGroupRequest_v0(Request): - API_KEY = 13 - API_VERSION = 0 - RESPONSE_TYPE = LeaveGroupResponse_v0 - SCHEMA = Schema( - ('group', String('utf-8')), - ('member_id', String('utf-8')) - ) - - -class LeaveGroupRequest_v1(Request): - API_KEY = 13 - API_VERSION = 1 - RESPONSE_TYPE = LeaveGroupResponse_v1 - SCHEMA = LeaveGroupRequest_v0.SCHEMA - - -LeaveGroupRequest = [LeaveGroupRequest_v0, LeaveGroupRequest_v1] -LeaveGroupResponse = [LeaveGroupResponse_v0, LeaveGroupResponse_v1] diff --git a/kafka/protocol/metadata.py b/kafka/protocol/metadata.py deleted file mode 100644 index 414e5b84..00000000 --- a/kafka/protocol/metadata.py +++ /dev/null @@ -1,200 +0,0 @@ -from __future__ import absolute_import - -from kafka.protocol.api import Request, Response -from kafka.protocol.types import Array, Boolean, Int16, Int32, Schema, String - - -class MetadataResponse_v0(Response): - API_KEY = 3 - API_VERSION = 0 - SCHEMA = Schema( - ('brokers', Array( - ('node_id', Int32), - ('host', String('utf-8')), - ('port', Int32))), - ('topics', Array( - ('error_code', Int16), - ('topic', String('utf-8')), - ('partitions', Array( - ('error_code', Int16), - ('partition', Int32), - ('leader', Int32), - ('replicas', Array(Int32)), - ('isr', Array(Int32)))))) - ) - - -class MetadataResponse_v1(Response): - API_KEY = 3 - API_VERSION = 1 - SCHEMA = Schema( - ('brokers', Array( - ('node_id', Int32), - ('host', String('utf-8')), - ('port', Int32), - ('rack', String('utf-8')))), - ('controller_id', Int32), - ('topics', Array( - ('error_code', Int16), - ('topic', String('utf-8')), - ('is_internal', Boolean), - ('partitions', Array( - ('error_code', Int16), - ('partition', Int32), - ('leader', Int32), - ('replicas', Array(Int32)), - ('isr', Array(Int32)))))) - ) - - -class MetadataResponse_v2(Response): - API_KEY = 3 - API_VERSION = 2 - SCHEMA = Schema( - ('brokers', Array( - ('node_id', Int32), - ('host', String('utf-8')), - ('port', Int32), - ('rack', String('utf-8')))), - ('cluster_id', String('utf-8')), # <-- Added cluster_id field in v2 - ('controller_id', Int32), - ('topics', Array( - ('error_code', Int16), - ('topic', String('utf-8')), - ('is_internal', Boolean), - ('partitions', Array( - ('error_code', Int16), - ('partition', Int32), - ('leader', Int32), - ('replicas', Array(Int32)), - ('isr', Array(Int32)))))) - ) - - -class MetadataResponse_v3(Response): - API_KEY = 3 - API_VERSION = 3 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('brokers', Array( - ('node_id', Int32), - ('host', String('utf-8')), - ('port', Int32), - ('rack', String('utf-8')))), - ('cluster_id', String('utf-8')), - ('controller_id', Int32), - ('topics', Array( - ('error_code', Int16), - ('topic', String('utf-8')), - ('is_internal', Boolean), - ('partitions', Array( - ('error_code', Int16), - ('partition', Int32), - ('leader', Int32), - ('replicas', Array(Int32)), - ('isr', Array(Int32)))))) - ) - - -class MetadataResponse_v4(Response): - API_KEY = 3 - API_VERSION = 4 - SCHEMA = MetadataResponse_v3.SCHEMA - - -class MetadataResponse_v5(Response): - API_KEY = 3 - API_VERSION = 5 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('brokers', Array( - ('node_id', Int32), - ('host', String('utf-8')), - ('port', Int32), - ('rack', String('utf-8')))), - ('cluster_id', String('utf-8')), - ('controller_id', Int32), - ('topics', Array( - ('error_code', Int16), - ('topic', String('utf-8')), - ('is_internal', Boolean), - ('partitions', Array( - ('error_code', Int16), - ('partition', Int32), - ('leader', Int32), - ('replicas', Array(Int32)), - ('isr', Array(Int32)), - ('offline_replicas', Array(Int32)))))) - ) - - -class MetadataRequest_v0(Request): - API_KEY = 3 - API_VERSION = 0 - RESPONSE_TYPE = MetadataResponse_v0 - SCHEMA = Schema( - ('topics', Array(String('utf-8'))) - ) - ALL_TOPICS = None # Empty Array (len 0) for topics returns all topics - - -class MetadataRequest_v1(Request): - API_KEY = 3 - API_VERSION = 1 - RESPONSE_TYPE = MetadataResponse_v1 - SCHEMA = MetadataRequest_v0.SCHEMA - ALL_TOPICS = -1 # Null Array (len -1) for topics returns all topics - NO_TOPICS = None # Empty array (len 0) for topics returns no topics - - -class MetadataRequest_v2(Request): - API_KEY = 3 - API_VERSION = 2 - RESPONSE_TYPE = MetadataResponse_v2 - SCHEMA = MetadataRequest_v1.SCHEMA - ALL_TOPICS = -1 # Null Array (len -1) for topics returns all topics - NO_TOPICS = None # Empty array (len 0) for topics returns no topics - - -class MetadataRequest_v3(Request): - API_KEY = 3 - API_VERSION = 3 - RESPONSE_TYPE = MetadataResponse_v3 - SCHEMA = MetadataRequest_v1.SCHEMA - ALL_TOPICS = -1 # Null Array (len -1) for topics returns all topics - NO_TOPICS = None # Empty array (len 0) for topics returns no topics - - -class MetadataRequest_v4(Request): - API_KEY = 3 - API_VERSION = 4 - RESPONSE_TYPE = MetadataResponse_v4 - SCHEMA = Schema( - ('topics', Array(String('utf-8'))), - ('allow_auto_topic_creation', Boolean) - ) - ALL_TOPICS = -1 # Null Array (len -1) for topics returns all topics - NO_TOPICS = None # Empty array (len 0) for topics returns no topics - - -class MetadataRequest_v5(Request): - """ - The v5 metadata request is the same as v4. - An additional field for offline_replicas has been added to the v5 metadata response - """ - API_KEY = 3 - API_VERSION = 5 - RESPONSE_TYPE = MetadataResponse_v5 - SCHEMA = MetadataRequest_v4.SCHEMA - ALL_TOPICS = -1 # Null Array (len -1) for topics returns all topics - NO_TOPICS = None # Empty array (len 0) for topics returns no topics - - -MetadataRequest = [ - MetadataRequest_v0, MetadataRequest_v1, MetadataRequest_v2, - MetadataRequest_v3, MetadataRequest_v4, MetadataRequest_v5 -] -MetadataResponse = [ - MetadataResponse_v0, MetadataResponse_v1, MetadataResponse_v2, - MetadataResponse_v3, MetadataResponse_v4, MetadataResponse_v5 -] diff --git a/kafka/protocol/offset.py b/kafka/protocol/offset.py deleted file mode 100644 index 1ed382b0..00000000 --- a/kafka/protocol/offset.py +++ /dev/null @@ -1,194 +0,0 @@ -from __future__ import absolute_import - -from kafka.protocol.api import Request, Response -from kafka.protocol.types import Array, Int8, Int16, Int32, Int64, Schema, String - -UNKNOWN_OFFSET = -1 - - -class OffsetResetStrategy(object): - LATEST = -1 - EARLIEST = -2 - NONE = 0 - - -class OffsetResponse_v0(Response): - API_KEY = 2 - API_VERSION = 0 - SCHEMA = Schema( - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('offsets', Array(Int64)))))) - ) - -class OffsetResponse_v1(Response): - API_KEY = 2 - API_VERSION = 1 - SCHEMA = Schema( - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('timestamp', Int64), - ('offset', Int64))))) - ) - - -class OffsetResponse_v2(Response): - API_KEY = 2 - API_VERSION = 2 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('timestamp', Int64), - ('offset', Int64))))) - ) - - -class OffsetResponse_v3(Response): - """ - on quota violation, brokers send out responses before throttling - """ - API_KEY = 2 - API_VERSION = 3 - SCHEMA = OffsetResponse_v2.SCHEMA - - -class OffsetResponse_v4(Response): - """ - Add leader_epoch to response - """ - API_KEY = 2 - API_VERSION = 4 - SCHEMA = Schema( - ('throttle_time_ms', Int32), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('timestamp', Int64), - ('offset', Int64), - ('leader_epoch', Int32))))) - ) - - -class OffsetResponse_v5(Response): - """ - adds a new error code, OFFSET_NOT_AVAILABLE - """ - API_KEY = 2 - API_VERSION = 5 - SCHEMA = OffsetResponse_v4.SCHEMA - - -class OffsetRequest_v0(Request): - API_KEY = 2 - API_VERSION = 0 - RESPONSE_TYPE = OffsetResponse_v0 - SCHEMA = Schema( - ('replica_id', Int32), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('timestamp', Int64), - ('max_offsets', Int32))))) - ) - DEFAULTS = { - 'replica_id': -1 - } - -class OffsetRequest_v1(Request): - API_KEY = 2 - API_VERSION = 1 - RESPONSE_TYPE = OffsetResponse_v1 - SCHEMA = Schema( - ('replica_id', Int32), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('timestamp', Int64))))) - ) - DEFAULTS = { - 'replica_id': -1 - } - - -class OffsetRequest_v2(Request): - API_KEY = 2 - API_VERSION = 2 - RESPONSE_TYPE = OffsetResponse_v2 - SCHEMA = Schema( - ('replica_id', Int32), - ('isolation_level', Int8), # <- added isolation_level - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('timestamp', Int64))))) - ) - DEFAULTS = { - 'replica_id': -1 - } - - -class OffsetRequest_v3(Request): - API_KEY = 2 - API_VERSION = 3 - RESPONSE_TYPE = OffsetResponse_v3 - SCHEMA = OffsetRequest_v2.SCHEMA - DEFAULTS = { - 'replica_id': -1 - } - - -class OffsetRequest_v4(Request): - """ - Add current_leader_epoch to request - """ - API_KEY = 2 - API_VERSION = 4 - RESPONSE_TYPE = OffsetResponse_v4 - SCHEMA = Schema( - ('replica_id', Int32), - ('isolation_level', Int8), # <- added isolation_level - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('current_leader_epoch', Int64), - ('timestamp', Int64))))) - ) - DEFAULTS = { - 'replica_id': -1 - } - - -class OffsetRequest_v5(Request): - API_KEY = 2 - API_VERSION = 5 - RESPONSE_TYPE = OffsetResponse_v5 - SCHEMA = OffsetRequest_v4.SCHEMA - DEFAULTS = { - 'replica_id': -1 - } - - -OffsetRequest = [ - OffsetRequest_v0, OffsetRequest_v1, OffsetRequest_v2, - OffsetRequest_v3, OffsetRequest_v4, OffsetRequest_v5, -] -OffsetResponse = [ - OffsetResponse_v0, OffsetResponse_v1, OffsetResponse_v2, - OffsetResponse_v3, OffsetResponse_v4, OffsetResponse_v5, -] diff --git a/kafka/protocol/produce.py b/kafka/protocol/produce.py deleted file mode 100644 index 9b3f6bf5..00000000 --- a/kafka/protocol/produce.py +++ /dev/null @@ -1,232 +0,0 @@ -from __future__ import absolute_import - -from kafka.protocol.api import Request, Response -from kafka.protocol.types import Int16, Int32, Int64, String, Array, Schema, Bytes - - -class ProduceResponse_v0(Response): - API_KEY = 0 - API_VERSION = 0 - SCHEMA = Schema( - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('offset', Int64))))) - ) - - -class ProduceResponse_v1(Response): - API_KEY = 0 - API_VERSION = 1 - SCHEMA = Schema( - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('offset', Int64))))), - ('throttle_time_ms', Int32) - ) - - -class ProduceResponse_v2(Response): - API_KEY = 0 - API_VERSION = 2 - SCHEMA = Schema( - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('offset', Int64), - ('timestamp', Int64))))), - ('throttle_time_ms', Int32) - ) - - -class ProduceResponse_v3(Response): - API_KEY = 0 - API_VERSION = 3 - SCHEMA = ProduceResponse_v2.SCHEMA - - -class ProduceResponse_v4(Response): - """ - The version number is bumped up to indicate that the client supports KafkaStorageException. - The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 3 - """ - API_KEY = 0 - API_VERSION = 4 - SCHEMA = ProduceResponse_v3.SCHEMA - - -class ProduceResponse_v5(Response): - API_KEY = 0 - API_VERSION = 5 - SCHEMA = Schema( - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('offset', Int64), - ('timestamp', Int64), - ('log_start_offset', Int64))))), - ('throttle_time_ms', Int32) - ) - - -class ProduceResponse_v6(Response): - """ - The version number is bumped to indicate that on quota violation brokers send out responses before throttling. - """ - API_KEY = 0 - API_VERSION = 6 - SCHEMA = ProduceResponse_v5.SCHEMA - - -class ProduceResponse_v7(Response): - """ - V7 bumped up to indicate ZStandard capability. (see KIP-110) - """ - API_KEY = 0 - API_VERSION = 7 - SCHEMA = ProduceResponse_v6.SCHEMA - - -class ProduceResponse_v8(Response): - """ - V8 bumped up to add two new fields record_errors offset list and error_message - (See KIP-467) - """ - API_KEY = 0 - API_VERSION = 8 - SCHEMA = Schema( - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('error_code', Int16), - ('offset', Int64), - ('timestamp', Int64), - ('log_start_offset', Int64)), - ('record_errors', (Array( - ('batch_index', Int32), - ('batch_index_error_message', String('utf-8')) - ))), - ('error_message', String('utf-8')) - ))), - ('throttle_time_ms', Int32) - ) - - -class ProduceRequest(Request): - API_KEY = 0 - - def expect_response(self): - if self.required_acks == 0: # pylint: disable=no-member - return False - return True - - -class ProduceRequest_v0(ProduceRequest): - API_VERSION = 0 - RESPONSE_TYPE = ProduceResponse_v0 - SCHEMA = Schema( - ('required_acks', Int16), - ('timeout', Int32), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('messages', Bytes))))) - ) - - -class ProduceRequest_v1(ProduceRequest): - API_VERSION = 1 - RESPONSE_TYPE = ProduceResponse_v1 - SCHEMA = ProduceRequest_v0.SCHEMA - - -class ProduceRequest_v2(ProduceRequest): - API_VERSION = 2 - RESPONSE_TYPE = ProduceResponse_v2 - SCHEMA = ProduceRequest_v1.SCHEMA - - -class ProduceRequest_v3(ProduceRequest): - API_VERSION = 3 - RESPONSE_TYPE = ProduceResponse_v3 - SCHEMA = Schema( - ('transactional_id', String('utf-8')), - ('required_acks', Int16), - ('timeout', Int32), - ('topics', Array( - ('topic', String('utf-8')), - ('partitions', Array( - ('partition', Int32), - ('messages', Bytes))))) - ) - - -class ProduceRequest_v4(ProduceRequest): - """ - The version number is bumped up to indicate that the client supports KafkaStorageException. - The KafkaStorageException will be translated to NotLeaderForPartitionException in the response if version <= 3 - """ - API_VERSION = 4 - RESPONSE_TYPE = ProduceResponse_v4 - SCHEMA = ProduceRequest_v3.SCHEMA - - -class ProduceRequest_v5(ProduceRequest): - """ - Same as v4. The version number is bumped since the v5 response includes an additional - partition level field: the log_start_offset. - """ - API_VERSION = 5 - RESPONSE_TYPE = ProduceResponse_v5 - SCHEMA = ProduceRequest_v4.SCHEMA - - -class ProduceRequest_v6(ProduceRequest): - """ - The version number is bumped to indicate that on quota violation brokers send out responses before throttling. - """ - API_VERSION = 6 - RESPONSE_TYPE = ProduceResponse_v6 - SCHEMA = ProduceRequest_v5.SCHEMA - - -class ProduceRequest_v7(ProduceRequest): - """ - V7 bumped up to indicate ZStandard capability. (see KIP-110) - """ - API_VERSION = 7 - RESPONSE_TYPE = ProduceResponse_v7 - SCHEMA = ProduceRequest_v6.SCHEMA - - -class ProduceRequest_v8(ProduceRequest): - """ - V8 bumped up to add two new fields record_errors offset list and error_message to PartitionResponse - (See KIP-467) - """ - API_VERSION = 8 - RESPONSE_TYPE = ProduceResponse_v8 - SCHEMA = ProduceRequest_v7.SCHEMA - - -ProduceRequest = [ - ProduceRequest_v0, ProduceRequest_v1, ProduceRequest_v2, - ProduceRequest_v3, ProduceRequest_v4, ProduceRequest_v5, - ProduceRequest_v6, ProduceRequest_v7, ProduceRequest_v8, -] -ProduceResponse = [ - ProduceResponse_v0, ProduceResponse_v1, ProduceResponse_v2, - ProduceResponse_v3, ProduceResponse_v4, ProduceResponse_v5, - ProduceResponse_v6, ProduceResponse_v7, ProduceResponse_v8, -] diff --git a/kafka/version.py b/kafka/version.py deleted file mode 100644 index 06306bd1..00000000 --- a/kafka/version.py +++ /dev/null @@ -1 +0,0 @@ -__version__ = '2.0.3-dev' diff --git a/tests/kafka/fixtures.py b/tests/kafka/fixtures.py index 76bde28f..b6854e54 100644 --- a/tests/kafka/fixtures.py +++ b/tests/kafka/fixtures.py @@ -15,8 +15,8 @@ from aiokafka import errors from aiokafka.errors import InvalidReplicationFactorError -from kafka.protocol.admin import CreateTopicsRequest -from kafka.protocol.metadata import MetadataRequest +from aiokafka.protocol.admin import CreateTopicsRequest +from aiokafka.protocol.metadata import MetadataRequest from tests.kafka.testutil import env_kafka_version, random_string from tests.kafka.service import ExternalService, SpawnedService diff --git a/tests/kafka/test_api_object_implementation.py b/tests/kafka/test_api_object_implementation.py deleted file mode 100644 index da80f148..00000000 --- a/tests/kafka/test_api_object_implementation.py +++ /dev/null @@ -1,18 +0,0 @@ -import abc -import pytest - -from kafka.protocol.api import Request -from kafka.protocol.api import Response - - -attr_names = [n for n in dir(Request) if isinstance(getattr(Request, n), abc.abstractproperty)] -@pytest.mark.parametrize('klass', Request.__subclasses__()) -@pytest.mark.parametrize('attr_name', attr_names) -def test_request_type_conformance(klass, attr_name): - assert hasattr(klass, attr_name) - -attr_names = [n for n in dir(Response) if isinstance(getattr(Response, n), abc.abstractproperty)] -@pytest.mark.parametrize('klass', Response.__subclasses__()) -@pytest.mark.parametrize('attr_name', attr_names) -def test_response_type_conformance(klass, attr_name): - assert hasattr(klass, attr_name) diff --git a/tests/kafka/test_object_conversion.py b/tests/kafka/test_object_conversion.py deleted file mode 100644 index 9b1ff213..00000000 --- a/tests/kafka/test_object_conversion.py +++ /dev/null @@ -1,236 +0,0 @@ -from kafka.protocol.admin import Request -from kafka.protocol.admin import Response -from kafka.protocol.types import Schema -from kafka.protocol.types import Array -from kafka.protocol.types import Int16 -from kafka.protocol.types import String - -import pytest - -@pytest.mark.parametrize('superclass', (Request, Response)) -class TestObjectConversion: - def test_get_item(self, superclass): - class TestClass(superclass): - API_KEY = 0 - API_VERSION = 0 - RESPONSE_TYPE = None # To satisfy the Request ABC - SCHEMA = Schema( - ('myobject', Int16)) - - tc = TestClass(myobject=0) - assert tc.get_item('myobject') == 0 - with pytest.raises(KeyError): - tc.get_item('does-not-exist') - - def test_with_empty_schema(self, superclass): - class TestClass(superclass): - API_KEY = 0 - API_VERSION = 0 - RESPONSE_TYPE = None # To satisfy the Request ABC - SCHEMA = Schema() - - tc = TestClass() - tc.encode() - assert tc.to_object() == {} - - def test_with_basic_schema(self, superclass): - class TestClass(superclass): - API_KEY = 0 - API_VERSION = 0 - RESPONSE_TYPE = None # To satisfy the Request ABC - SCHEMA = Schema( - ('myobject', Int16)) - - tc = TestClass(myobject=0) - tc.encode() - assert tc.to_object() == {'myobject': 0} - - def test_with_basic_array_schema(self, superclass): - class TestClass(superclass): - API_KEY = 0 - API_VERSION = 0 - RESPONSE_TYPE = None # To satisfy the Request ABC - SCHEMA = Schema( - ('myarray', Array(Int16))) - - tc = TestClass(myarray=[1,2,3]) - tc.encode() - assert tc.to_object()['myarray'] == [1, 2, 3] - - def test_with_complex_array_schema(self, superclass): - class TestClass(superclass): - API_KEY = 0 - API_VERSION = 0 - RESPONSE_TYPE = None # To satisfy the Request ABC - SCHEMA = Schema( - ('myarray', Array( - ('subobject', Int16), - ('othersubobject', String('utf-8'))))) - - tc = TestClass( - myarray=[[10, 'hello']] - ) - tc.encode() - obj = tc.to_object() - assert len(obj['myarray']) == 1 - assert obj['myarray'][0]['subobject'] == 10 - assert obj['myarray'][0]['othersubobject'] == 'hello' - - def test_with_array_and_other(self, superclass): - class TestClass(superclass): - API_KEY = 0 - API_VERSION = 0 - RESPONSE_TYPE = None # To satisfy the Request ABC - SCHEMA = Schema( - ('myarray', Array( - ('subobject', Int16), - ('othersubobject', String('utf-8')))), - ('notarray', Int16)) - - tc = TestClass( - myarray=[[10, 'hello']], - notarray=42 - ) - - obj = tc.to_object() - assert len(obj['myarray']) == 1 - assert obj['myarray'][0]['subobject'] == 10 - assert obj['myarray'][0]['othersubobject'] == 'hello' - assert obj['notarray'] == 42 - - def test_with_nested_array(self, superclass): - class TestClass(superclass): - API_KEY = 0 - API_VERSION = 0 - RESPONSE_TYPE = None # To satisfy the Request ABC - SCHEMA = Schema( - ('myarray', Array( - ('subarray', Array(Int16)), - ('otherobject', Int16)))) - - tc = TestClass( - myarray=[ - [[1, 2], 2], - [[2, 3], 4], - ] - ) - print(tc.encode()) - - - obj = tc.to_object() - assert len(obj['myarray']) == 2 - assert obj['myarray'][0]['subarray'] == [1, 2] - assert obj['myarray'][0]['otherobject'] == 2 - assert obj['myarray'][1]['subarray'] == [2, 3] - assert obj['myarray'][1]['otherobject'] == 4 - - def test_with_complex_nested_array(self, superclass): - class TestClass(superclass): - API_KEY = 0 - API_VERSION = 0 - RESPONSE_TYPE = None # To satisfy the Request ABC - SCHEMA = Schema( - ('myarray', Array( - ('subarray', Array( - ('innertest', String('utf-8')), - ('otherinnertest', String('utf-8')))), - ('othersubarray', Array(Int16)))), - ('notarray', String('utf-8'))) - - tc = TestClass( - myarray=[ - [[['hello', 'hello'], ['hello again', 'hello again']], [0]], - [[['hello', 'hello again']], [1]], - ], - notarray='notarray' - ) - tc.encode() - - obj = tc.to_object() - - assert obj['notarray'] == 'notarray' - myarray = obj['myarray'] - assert len(myarray) == 2 - - assert myarray[0]['othersubarray'] == [0] - assert len(myarray[0]['subarray']) == 2 - assert myarray[0]['subarray'][0]['innertest'] == 'hello' - assert myarray[0]['subarray'][0]['otherinnertest'] == 'hello' - assert myarray[0]['subarray'][1]['innertest'] == 'hello again' - assert myarray[0]['subarray'][1]['otherinnertest'] == 'hello again' - - assert myarray[1]['othersubarray'] == [1] - assert len(myarray[1]['subarray']) == 1 - assert myarray[1]['subarray'][0]['innertest'] == 'hello' - assert myarray[1]['subarray'][0]['otherinnertest'] == 'hello again' - -def test_with_metadata_response(): - from kafka.protocol.metadata import MetadataResponse_v5 - tc = MetadataResponse_v5( - throttle_time_ms=0, - brokers=[ - [0, 'testhost0', 9092, 'testrack0'], - [1, 'testhost1', 9092, 'testrack1'], - ], - cluster_id='abcd', - controller_id=0, - topics=[ - [0, 'testtopic1', False, [ - [0, 0, 0, [0, 1], [0, 1], []], - [0, 1, 1, [1, 0], [1, 0], []], - ], - ], [0, 'other-test-topic', True, [ - [0, 0, 0, [0, 1], [0, 1], []], - ] - ]] - ) - tc.encode() # Make sure this object encodes successfully - - - obj = tc.to_object() - - assert obj['throttle_time_ms'] == 0 - - assert len(obj['brokers']) == 2 - assert obj['brokers'][0]['node_id'] == 0 - assert obj['brokers'][0]['host'] == 'testhost0' - assert obj['brokers'][0]['port'] == 9092 - assert obj['brokers'][0]['rack'] == 'testrack0' - assert obj['brokers'][1]['node_id'] == 1 - assert obj['brokers'][1]['host'] == 'testhost1' - assert obj['brokers'][1]['port'] == 9092 - assert obj['brokers'][1]['rack'] == 'testrack1' - - assert obj['cluster_id'] == 'abcd' - assert obj['controller_id'] == 0 - - assert len(obj['topics']) == 2 - assert obj['topics'][0]['error_code'] == 0 - assert obj['topics'][0]['topic'] == 'testtopic1' - assert obj['topics'][0]['is_internal'] == False - assert len(obj['topics'][0]['partitions']) == 2 - assert obj['topics'][0]['partitions'][0]['error_code'] == 0 - assert obj['topics'][0]['partitions'][0]['partition'] == 0 - assert obj['topics'][0]['partitions'][0]['leader'] == 0 - assert obj['topics'][0]['partitions'][0]['replicas'] == [0, 1] - assert obj['topics'][0]['partitions'][0]['isr'] == [0, 1] - assert obj['topics'][0]['partitions'][0]['offline_replicas'] == [] - assert obj['topics'][0]['partitions'][1]['error_code'] == 0 - assert obj['topics'][0]['partitions'][1]['partition'] == 1 - assert obj['topics'][0]['partitions'][1]['leader'] == 1 - assert obj['topics'][0]['partitions'][1]['replicas'] == [1, 0] - assert obj['topics'][0]['partitions'][1]['isr'] == [1, 0] - assert obj['topics'][0]['partitions'][1]['offline_replicas'] == [] - - assert obj['topics'][1]['error_code'] == 0 - assert obj['topics'][1]['topic'] == 'other-test-topic' - assert obj['topics'][1]['is_internal'] == True - assert len(obj['topics'][1]['partitions']) == 1 - assert obj['topics'][1]['partitions'][0]['error_code'] == 0 - assert obj['topics'][1]['partitions'][0]['partition'] == 0 - assert obj['topics'][1]['partitions'][0]['leader'] == 0 - assert obj['topics'][1]['partitions'][0]['replicas'] == [0, 1] - assert obj['topics'][1]['partitions'][0]['isr'] == [0, 1] - assert obj['topics'][1]['partitions'][0]['offline_replicas'] == [] - - tc.encode() diff --git a/tests/kafka/test_protocol.py b/tests/kafka/test_protocol.py deleted file mode 100644 index 6a77e19d..00000000 --- a/tests/kafka/test_protocol.py +++ /dev/null @@ -1,336 +0,0 @@ -#pylint: skip-file -import io -import struct - -import pytest - -from kafka.protocol.api import RequestHeader -from kafka.protocol.commit import GroupCoordinatorRequest -from kafka.protocol.fetch import FetchRequest, FetchResponse -from kafka.protocol.message import Message, MessageSet, PartialMessage -from kafka.protocol.metadata import MetadataRequest -from kafka.protocol.types import Int16, Int32, Int64, String, UnsignedVarInt32, CompactString, CompactArray, CompactBytes - - -def test_create_message(): - payload = b'test' - key = b'key' - msg = Message(payload, key=key) - assert msg.magic == 0 - assert msg.attributes == 0 - assert msg.key == key - assert msg.value == payload - - -def test_encode_message_v0(): - message = Message(b'test', key=b'key') - encoded = message.encode() - expect = b''.join([ - struct.pack('>i', -1427009701), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 3), # Length of key - b'key', # key - struct.pack('>i', 4), # Length of value - b'test', # value - ]) - assert encoded == expect - - -def test_encode_message_v1(): - message = Message(b'test', key=b'key', magic=1, timestamp=1234) - encoded = message.encode() - expect = b''.join([ - struct.pack('>i', 1331087195), # CRC - struct.pack('>bb', 1, 0), # Magic, flags - struct.pack('>q', 1234), # Timestamp - struct.pack('>i', 3), # Length of key - b'key', # key - struct.pack('>i', 4), # Length of value - b'test', # value - ]) - assert encoded == expect - - -def test_decode_message(): - encoded = b''.join([ - struct.pack('>i', -1427009701), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 3), # Length of key - b'key', # key - struct.pack('>i', 4), # Length of value - b'test', # value - ]) - decoded_message = Message.decode(encoded) - msg = Message(b'test', key=b'key') - msg.encode() # crc is recalculated during encoding - assert decoded_message == msg - - -def test_decode_message_validate_crc(): - encoded = b''.join([ - struct.pack('>i', -1427009701), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 3), # Length of key - b'key', # key - struct.pack('>i', 4), # Length of value - b'test', # value - ]) - decoded_message = Message.decode(encoded) - assert decoded_message.validate_crc() is True - - encoded = b''.join([ - struct.pack('>i', 1234), # Incorrect CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 3), # Length of key - b'key', # key - struct.pack('>i', 4), # Length of value - b'test', # value - ]) - decoded_message = Message.decode(encoded) - assert decoded_message.validate_crc() is False - - -def test_encode_message_set(): - messages = [ - Message(b'v1', key=b'k1'), - Message(b'v2', key=b'k2') - ] - encoded = MessageSet.encode([(0, msg.encode()) - for msg in messages]) - expect = b''.join([ - struct.pack('>q', 0), # MsgSet Offset - struct.pack('>i', 18), # Msg Size - struct.pack('>i', 1474775406), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k1', # Key - struct.pack('>i', 2), # Length of value - b'v1', # Value - - struct.pack('>q', 0), # MsgSet Offset - struct.pack('>i', 18), # Msg Size - struct.pack('>i', -16383415), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k2', # Key - struct.pack('>i', 2), # Length of value - b'v2', # Value - ]) - expect = struct.pack('>i', len(expect)) + expect - assert encoded == expect - - -def test_decode_message_set(): - encoded = b''.join([ - struct.pack('>q', 0), # MsgSet Offset - struct.pack('>i', 18), # Msg Size - struct.pack('>i', 1474775406), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k1', # Key - struct.pack('>i', 2), # Length of value - b'v1', # Value - - struct.pack('>q', 1), # MsgSet Offset - struct.pack('>i', 18), # Msg Size - struct.pack('>i', -16383415), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k2', # Key - struct.pack('>i', 2), # Length of value - b'v2', # Value - ]) - - msgs = MessageSet.decode(encoded, bytes_to_read=len(encoded)) - assert len(msgs) == 2 - msg1, msg2 = msgs - - returned_offset1, message1_size, decoded_message1 = msg1 - returned_offset2, message2_size, decoded_message2 = msg2 - - assert returned_offset1 == 0 - message1 = Message(b'v1', key=b'k1') - message1.encode() - assert decoded_message1 == message1 - - assert returned_offset2 == 1 - message2 = Message(b'v2', key=b'k2') - message2.encode() - assert decoded_message2 == message2 - - -def test_encode_message_header(): - expect = b''.join([ - struct.pack('>h', 10), # API Key - struct.pack('>h', 0), # API Version - struct.pack('>i', 4), # Correlation Id - struct.pack('>h', len('client3')), # Length of clientId - b'client3', # ClientId - ]) - - req = GroupCoordinatorRequest[0]('foo') - header = RequestHeader(req, correlation_id=4, client_id='client3') - assert header.encode() == expect - - -def test_decode_message_set_partial(): - encoded = b''.join([ - struct.pack('>q', 0), # Msg Offset - struct.pack('>i', 18), # Msg Size - struct.pack('>i', 1474775406), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k1', # Key - struct.pack('>i', 2), # Length of value - b'v1', # Value - - struct.pack('>q', 1), # Msg Offset - struct.pack('>i', 24), # Msg Size (larger than remaining MsgSet size) - struct.pack('>i', -16383415), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k2', # Key - struct.pack('>i', 8), # Length of value - b'ar', # Value (truncated) - ]) - - msgs = MessageSet.decode(encoded, bytes_to_read=len(encoded)) - assert len(msgs) == 2 - msg1, msg2 = msgs - - returned_offset1, message1_size, decoded_message1 = msg1 - returned_offset2, message2_size, decoded_message2 = msg2 - - assert returned_offset1 == 0 - message1 = Message(b'v1', key=b'k1') - message1.encode() - assert decoded_message1 == message1 - - assert returned_offset2 is None - assert message2_size is None - assert decoded_message2 == PartialMessage() - - -def test_decode_fetch_response_partial(): - encoded = b''.join([ - Int32.encode(1), # Num Topics (Array) - String('utf-8').encode('foobar'), - Int32.encode(2), # Num Partitions (Array) - Int32.encode(0), # Partition id - Int16.encode(0), # Error Code - Int64.encode(1234), # Highwater offset - Int32.encode(52), # MessageSet size - Int64.encode(0), # Msg Offset - Int32.encode(18), # Msg Size - struct.pack('>i', 1474775406), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k1', # Key - struct.pack('>i', 2), # Length of value - b'v1', # Value - - Int64.encode(1), # Msg Offset - struct.pack('>i', 24), # Msg Size (larger than remaining MsgSet size) - struct.pack('>i', -16383415), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k2', # Key - struct.pack('>i', 8), # Length of value - b'ar', # Value (truncated) - Int32.encode(1), - Int16.encode(0), - Int64.encode(2345), - Int32.encode(52), # MessageSet size - Int64.encode(0), # Msg Offset - Int32.encode(18), # Msg Size - struct.pack('>i', 1474775406), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k1', # Key - struct.pack('>i', 2), # Length of value - b'v1', # Value - - Int64.encode(1), # Msg Offset - struct.pack('>i', 24), # Msg Size (larger than remaining MsgSet size) - struct.pack('>i', -16383415), # CRC - struct.pack('>bb', 0, 0), # Magic, flags - struct.pack('>i', 2), # Length of key - b'k2', # Key - struct.pack('>i', 8), # Length of value - b'ar', # Value (truncated) - ]) - resp = FetchResponse[0].decode(io.BytesIO(encoded)) - assert len(resp.topics) == 1 - topic, partitions = resp.topics[0] - assert topic == 'foobar' - assert len(partitions) == 2 - - m1 = MessageSet.decode( - partitions[0][3], bytes_to_read=len(partitions[0][3])) - assert len(m1) == 2 - assert m1[1] == (None, None, PartialMessage()) - - -def test_struct_unrecognized_kwargs(): - try: - mr = MetadataRequest[0](topicz='foo') - assert False, 'Structs should not allow unrecognized kwargs' - except ValueError: - pass - - -def test_struct_missing_kwargs(): - fr = FetchRequest[0](max_wait_time=100) - assert fr.min_bytes is None - - -def test_unsigned_varint_serde(): - pairs = { - 0: [0], - -1: [0xff, 0xff, 0xff, 0xff, 0x0f], - 1: [1], - 63: [0x3f], - -64: [0xc0, 0xff, 0xff, 0xff, 0x0f], - 64: [0x40], - 8191: [0xff, 0x3f], - -8192: [0x80, 0xc0, 0xff, 0xff, 0x0f], - 8192: [0x80, 0x40], - -8193: [0xff, 0xbf, 0xff, 0xff, 0x0f], - 1048575: [0xff, 0xff, 0x3f], - - } - for value, expected_encoded in pairs.items(): - value &= 0xffffffff - encoded = UnsignedVarInt32.encode(value) - assert encoded == b''.join(struct.pack('>B', x) for x in expected_encoded) - assert value == UnsignedVarInt32.decode(io.BytesIO(encoded)) - - -def test_compact_data_structs(): - cs = CompactString() - encoded = cs.encode(None) - assert encoded == struct.pack('B', 0) - decoded = cs.decode(io.BytesIO(encoded)) - assert decoded is None - assert b'\x01' == cs.encode('') - assert '' == cs.decode(io.BytesIO(b'\x01')) - encoded = cs.encode("foobarbaz") - assert cs.decode(io.BytesIO(encoded)) == "foobarbaz" - - arr = CompactArray(CompactString()) - assert arr.encode(None) == b'\x00' - assert arr.decode(io.BytesIO(b'\x00')) is None - enc = arr.encode([]) - assert enc == b'\x01' - assert [] == arr.decode(io.BytesIO(enc)) - encoded = arr.encode(["foo", "bar", "baz", "quux"]) - assert arr.decode(io.BytesIO(encoded)) == ["foo", "bar", "baz", "quux"] - - enc = CompactBytes.encode(None) - assert enc == b'\x00' - assert CompactBytes.decode(io.BytesIO(b'\x00')) is None - enc = CompactBytes.encode(b'') - assert enc == b'\x01' - assert CompactBytes.decode(io.BytesIO(b'\x01')) is b'' - enc = CompactBytes.encode(b'foo') - assert CompactBytes.decode(io.BytesIO(enc)) == b'foo' diff --git a/tests/test_client.py b/tests/test_client.py index e9ceb517..f17ca673 100644 --- a/tests/test_client.py +++ b/tests/test_client.py @@ -4,11 +4,6 @@ from typing import Any from unittest import mock -from kafka.protocol.metadata import ( - MetadataRequest_v0 as MetadataRequest, - MetadataResponse_v0 as MetadataResponse) -from kafka.protocol.fetch import FetchRequest_v0 - from aiokafka import __version__ from aiokafka.client import AIOKafkaClient, ConnectionGroup, CoordinationType from aiokafka.conn import AIOKafkaConnection, CloseReason @@ -16,6 +11,10 @@ KafkaError, KafkaConnectionError, RequestTimedOutError, NodeNotReadyError, UnrecognizedBrokerVersion ) +from aiokafka.protocol.metadata import ( + MetadataRequest_v0 as MetadataRequest, + MetadataResponse_v0 as MetadataResponse) +from aiokafka.protocol.fetch import FetchRequest_v0 from aiokafka.util import create_task, get_running_loop from ._testutil import ( KafkaIntegrationTestCase, run_until_complete, kafka_versions diff --git a/tests/test_cluster.py b/tests/test_cluster.py index 0fad6e31..84ff1549 100644 --- a/tests/test_cluster.py +++ b/tests/test_cluster.py @@ -1,6 +1,5 @@ -from kafka.protocol.metadata import MetadataResponse - from aiokafka.cluster import ClusterMetadata +from aiokafka.protocol.metadata import MetadataResponse def test_empty_broker_list(): diff --git a/tests/test_conn.py b/tests/test_conn.py index 3862c11c..a1cedc92 100644 --- a/tests/test_conn.py +++ b/tests/test_conn.py @@ -5,23 +5,22 @@ from typing import Any from unittest import mock -from kafka.protocol.metadata import ( +from aiokafka.conn import AIOKafkaConnection, create_conn, VersionInfo +from aiokafka.errors import ( + KafkaConnectionError, CorrelationIdError, KafkaError, NoError, + UnknownError, UnsupportedSaslMechanismError, IllegalSaslStateError +) +from aiokafka.protocol.metadata import ( MetadataRequest_v0 as MetadataRequest, MetadataResponse_v0 as MetadataResponse) -from kafka.protocol.commit import ( +from aiokafka.protocol.commit import ( GroupCoordinatorRequest_v0 as GroupCoordinatorRequest, GroupCoordinatorResponse_v0 as GroupCoordinatorResponse) -from kafka.protocol.admin import ( +from aiokafka.protocol.admin import ( SaslHandShakeRequest, SaslHandShakeResponse, SaslAuthenticateRequest, SaslAuthenticateResponse ) -from kafka.protocol.produce import ProduceRequest_v0 as ProduceRequest - -from aiokafka.conn import AIOKafkaConnection, create_conn, VersionInfo -from aiokafka.errors import ( - KafkaConnectionError, CorrelationIdError, KafkaError, NoError, - UnknownError, UnsupportedSaslMechanismError, IllegalSaslStateError -) +from aiokafka.protocol.produce import ProduceRequest_v0 as ProduceRequest from aiokafka.record.legacy_records import LegacyRecordBatchBuilder from ._testutil import KafkaIntegrationTestCase, run_until_complete from aiokafka.util import get_running_loop diff --git a/tests/test_coordinator.py b/tests/test_coordinator.py index 59b26de9..38bf12d9 100644 --- a/tests/test_coordinator.py +++ b/tests/test_coordinator.py @@ -2,28 +2,27 @@ import re from unittest import mock -from kafka.protocol.group import ( +from aiokafka import ConsumerRebalanceListener +from aiokafka.client import AIOKafkaClient +import aiokafka.errors as Errors +from aiokafka.structs import OffsetAndMetadata, TopicPartition +from aiokafka.consumer.group_coordinator import ( + GroupCoordinator, CoordinatorGroupRebalance, NoGroupCoordinator) +from aiokafka.consumer.subscription_state import SubscriptionState +from aiokafka.protocol.group import ( JoinGroupRequest_v0 as JoinGroupRequest, SyncGroupResponse_v0 as SyncGroupResponse, LeaveGroupRequest_v0 as LeaveGroupRequest, HeartbeatRequest_v0 as HeartbeatRequest, ) -from kafka.protocol.commit import ( +from aiokafka.protocol.commit import ( OffsetCommitRequest, OffsetCommitResponse_v2, OffsetFetchRequest_v1 as OffsetFetchRequest ) +from aiokafka.util import create_future, create_task, get_running_loop from ._testutil import KafkaIntegrationTestCase, run_until_complete -from aiokafka import ConsumerRebalanceListener -from aiokafka.client import AIOKafkaClient -import aiokafka.errors as Errors -from aiokafka.structs import OffsetAndMetadata, TopicPartition -from aiokafka.consumer.group_coordinator import ( - GroupCoordinator, CoordinatorGroupRebalance, NoGroupCoordinator) -from aiokafka.consumer.subscription_state import SubscriptionState -from aiokafka.util import create_future, create_task, get_running_loop - UNKNOWN_MEMBER_ID = JoinGroupRequest.UNKNOWN_MEMBER_ID diff --git a/tests/test_fetcher.py b/tests/test_fetcher.py index c6d8e23f..aff4d69c 100644 --- a/tests/test_fetcher.py +++ b/tests/test_fetcher.py @@ -3,30 +3,30 @@ import unittest from unittest import mock -from kafka.protocol.fetch import ( +from aiokafka.client import AIOKafkaClient +from aiokafka.consumer.fetcher import ( + Fetcher, FetchResult, FetchError, ConsumerRecord, OffsetResetStrategy, + PartitionRecords, READ_COMMITTED, READ_UNCOMMITTED +) +from aiokafka.consumer.subscription_state import SubscriptionState +from aiokafka.errors import ( + TopicAuthorizationFailedError, UnknownError, UnknownTopicOrPartitionError, + OffsetOutOfRangeError, KafkaTimeoutError, NotLeaderForPartitionError +) +from aiokafka.protocol.fetch import ( FetchRequest_v0 as FetchRequest, FetchResponse_v0 as FetchResponse ) -from kafka.protocol.offset import OffsetResponse +from aiokafka.protocol.offset import OffsetResponse from aiokafka.record.legacy_records import LegacyRecordBatchBuilder from aiokafka.record.default_records import ( # NB: test_solitary_abort_marker relies on implementation details _DefaultRecordBatchBuilderPy as DefaultRecordBatchBuilder) from aiokafka.record.memory_records import MemoryRecords - -from aiokafka.errors import ( - TopicAuthorizationFailedError, UnknownError, UnknownTopicOrPartitionError, - OffsetOutOfRangeError, KafkaTimeoutError, NotLeaderForPartitionError -) from aiokafka.structs import ( TopicPartition, OffsetAndTimestamp, OffsetAndMetadata ) -from aiokafka.client import AIOKafkaClient -from aiokafka.consumer.fetcher import ( - Fetcher, FetchResult, FetchError, ConsumerRecord, OffsetResetStrategy, - PartitionRecords, READ_COMMITTED, READ_UNCOMMITTED -) -from aiokafka.consumer.subscription_state import SubscriptionState from aiokafka.util import create_future, create_task, get_running_loop + from ._testutil import run_until_complete diff --git a/tests/test_producer.py b/tests/test_producer.py index 9cc6858b..a6004a4e 100644 --- a/tests/test_producer.py +++ b/tests/test_producer.py @@ -6,9 +6,6 @@ import weakref from unittest import mock -from kafka.protocol.produce import ProduceResponse - -from aiokafka.producer import AIOKafkaProducer from aiokafka.client import AIOKafkaClient from aiokafka.cluster import ClusterMetadata from aiokafka.consumer import AIOKafkaConsumer @@ -17,6 +14,8 @@ MessageSizeTooLargeError, NotLeaderForPartitionError, LeaderNotAvailableError, RequestTimedOutError, UnsupportedVersionError, ProducerClosed, KafkaError) +from aiokafka.producer import AIOKafkaProducer +from aiokafka.protocol.produce import ProduceResponse from aiokafka.util import create_future from ._testutil import ( diff --git a/tests/test_protocol.py b/tests/test_protocol.py new file mode 100644 index 00000000..240ca356 --- /dev/null +++ b/tests/test_protocol.py @@ -0,0 +1,376 @@ +import abc +import io +import struct + +import pytest + +from aiokafka.protocol.api import RequestHeader, Request, Response +from aiokafka.protocol.commit import GroupCoordinatorRequest +from aiokafka.protocol.fetch import FetchRequest, FetchResponse +from aiokafka.protocol.message import Message, MessageSet, PartialMessage +from aiokafka.protocol.metadata import MetadataRequest +from aiokafka.protocol.types import ( + Int16, + Int32, + Int64, + String, + UnsignedVarInt32, + CompactString, + CompactArray, + CompactBytes, +) + + +def test_create_message(): + payload = b"test" + key = b"key" + msg = Message(payload, key=key) + assert msg.magic == 0 + assert msg.attributes == 0 + assert msg.key == key + assert msg.value == payload + + +def test_encode_message_v0(): + message = Message(b"test", key=b"key") + encoded = message.encode() + expect = b"".join( + [ + struct.pack(">i", -1427009701), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 3), # Length of key + b"key", # key + struct.pack(">i", 4), # Length of value + b"test", # value + ] + ) + assert encoded == expect + + +def test_encode_message_v1(): + message = Message(b"test", key=b"key", magic=1, timestamp=1234) + encoded = message.encode() + expect = b"".join( + [ + struct.pack(">i", 1331087195), # CRC + struct.pack(">bb", 1, 0), # Magic, flags + struct.pack(">q", 1234), # Timestamp + struct.pack(">i", 3), # Length of key + b"key", # key + struct.pack(">i", 4), # Length of value + b"test", # value + ] + ) + assert encoded == expect + + +def test_decode_message(): + encoded = b"".join( + [ + struct.pack(">i", -1427009701), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 3), # Length of key + b"key", # key + struct.pack(">i", 4), # Length of value + b"test", # value + ] + ) + decoded_message = Message.decode(encoded) + msg = Message(b"test", key=b"key") + msg.encode() # crc is recalculated during encoding + assert decoded_message == msg + + +def test_decode_message_validate_crc(): + encoded = b"".join( + [ + struct.pack(">i", -1427009701), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 3), # Length of key + b"key", # key + struct.pack(">i", 4), # Length of value + b"test", # value + ] + ) + decoded_message = Message.decode(encoded) + assert decoded_message.validate_crc() is True + + encoded = b"".join( + [ + struct.pack(">i", 1234), # Incorrect CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 3), # Length of key + b"key", # key + struct.pack(">i", 4), # Length of value + b"test", # value + ] + ) + decoded_message = Message.decode(encoded) + assert decoded_message.validate_crc() is False + + +def test_encode_message_set(): + messages = [Message(b"v1", key=b"k1"), Message(b"v2", key=b"k2")] + encoded = MessageSet.encode([(0, msg.encode()) for msg in messages]) + expect = b"".join( + [ + struct.pack(">q", 0), # MsgSet Offset + struct.pack(">i", 18), # Msg Size + struct.pack(">i", 1474775406), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 2), # Length of key + b"k1", # Key + struct.pack(">i", 2), # Length of value + b"v1", # Value + struct.pack(">q", 0), # MsgSet Offset + struct.pack(">i", 18), # Msg Size + struct.pack(">i", -16383415), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 2), # Length of key + b"k2", # Key + struct.pack(">i", 2), # Length of value + b"v2", # Value + ] + ) + expect = struct.pack(">i", len(expect)) + expect + assert encoded == expect + + +def test_decode_message_set(): + encoded = b"".join( + [ + struct.pack(">q", 0), # MsgSet Offset + struct.pack(">i", 18), # Msg Size + struct.pack(">i", 1474775406), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 2), # Length of key + b"k1", # Key + struct.pack(">i", 2), # Length of value + b"v1", # Value + struct.pack(">q", 1), # MsgSet Offset + struct.pack(">i", 18), # Msg Size + struct.pack(">i", -16383415), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 2), # Length of key + b"k2", # Key + struct.pack(">i", 2), # Length of value + b"v2", # Value + ] + ) + + msgs = MessageSet.decode(encoded, bytes_to_read=len(encoded)) + assert len(msgs) == 2 + msg1, msg2 = msgs + + returned_offset1, message1_size, decoded_message1 = msg1 + returned_offset2, message2_size, decoded_message2 = msg2 + + assert returned_offset1 == 0 + message1 = Message(b"v1", key=b"k1") + message1.encode() + assert decoded_message1 == message1 + + assert returned_offset2 == 1 + message2 = Message(b"v2", key=b"k2") + message2.encode() + assert decoded_message2 == message2 + + +def test_encode_message_header(): + expect = b"".join( + [ + struct.pack(">h", 10), # API Key + struct.pack(">h", 0), # API Version + struct.pack(">i", 4), # Correlation Id + struct.pack(">h", len("client3")), # Length of clientId + b"client3", # ClientId + ] + ) + + req = GroupCoordinatorRequest[0]("foo") + header = RequestHeader(req, correlation_id=4, client_id="client3") + assert header.encode() == expect + + +def test_decode_message_set_partial(): + encoded = b"".join( + [ + struct.pack(">q", 0), # Msg Offset + struct.pack(">i", 18), # Msg Size + struct.pack(">i", 1474775406), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 2), # Length of key + b"k1", # Key + struct.pack(">i", 2), # Length of value + b"v1", # Value + struct.pack(">q", 1), # Msg Offset + struct.pack(">i", 24), # Msg Size (larger than remaining MsgSet size) + struct.pack(">i", -16383415), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 2), # Length of key + b"k2", # Key + struct.pack(">i", 8), # Length of value + b"ar", # Value (truncated) + ] + ) + + msgs = MessageSet.decode(encoded, bytes_to_read=len(encoded)) + assert len(msgs) == 2 + msg1, msg2 = msgs + + returned_offset1, message1_size, decoded_message1 = msg1 + returned_offset2, message2_size, decoded_message2 = msg2 + + assert returned_offset1 == 0 + message1 = Message(b"v1", key=b"k1") + message1.encode() + assert decoded_message1 == message1 + + assert returned_offset2 is None + assert message2_size is None + assert decoded_message2 == PartialMessage() + + +def test_decode_fetch_response_partial(): + encoded = b"".join( + [ + Int32.encode(1), # Num Topics (Array) + String("utf-8").encode("foobar"), + Int32.encode(2), # Num Partitions (Array) + Int32.encode(0), # Partition id + Int16.encode(0), # Error Code + Int64.encode(1234), # Highwater offset + Int32.encode(52), # MessageSet size + Int64.encode(0), # Msg Offset + Int32.encode(18), # Msg Size + struct.pack(">i", 1474775406), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 2), # Length of key + b"k1", # Key + struct.pack(">i", 2), # Length of value + b"v1", # Value + Int64.encode(1), # Msg Offset + struct.pack(">i", 24), # Msg Size (larger than remaining MsgSet size) + struct.pack(">i", -16383415), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 2), # Length of key + b"k2", # Key + struct.pack(">i", 8), # Length of value + b"ar", # Value (truncated) + Int32.encode(1), + Int16.encode(0), + Int64.encode(2345), + Int32.encode(52), # MessageSet size + Int64.encode(0), # Msg Offset + Int32.encode(18), # Msg Size + struct.pack(">i", 1474775406), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 2), # Length of key + b"k1", # Key + struct.pack(">i", 2), # Length of value + b"v1", # Value + Int64.encode(1), # Msg Offset + struct.pack(">i", 24), # Msg Size (larger than remaining MsgSet size) + struct.pack(">i", -16383415), # CRC + struct.pack(">bb", 0, 0), # Magic, flags + struct.pack(">i", 2), # Length of key + b"k2", # Key + struct.pack(">i", 8), # Length of value + b"ar", # Value (truncated) + ] + ) + resp = FetchResponse[0].decode(io.BytesIO(encoded)) + assert len(resp.topics) == 1 + topic, partitions = resp.topics[0] + assert topic == "foobar" + assert len(partitions) == 2 + + m1 = MessageSet.decode(partitions[0][3], bytes_to_read=len(partitions[0][3])) + assert len(m1) == 2 + assert m1[1] == (None, None, PartialMessage()) + + +def test_struct_unrecognized_kwargs(): + try: + MetadataRequest[0](topicz="foo") + assert False, "Structs should not allow unrecognized kwargs" + except ValueError: + pass + + +def test_struct_missing_kwargs(): + fr = FetchRequest[0](max_wait_time=100) + assert fr.min_bytes is None + + +def test_unsigned_varint_serde(): + pairs = { + 0: [0], + -1: [0xFF, 0xFF, 0xFF, 0xFF, 0x0F], + 1: [1], + 63: [0x3F], + -64: [0xC0, 0xFF, 0xFF, 0xFF, 0x0F], + 64: [0x40], + 8191: [0xFF, 0x3F], + -8192: [0x80, 0xC0, 0xFF, 0xFF, 0x0F], + 8192: [0x80, 0x40], + -8193: [0xFF, 0xBF, 0xFF, 0xFF, 0x0F], + 1048575: [0xFF, 0xFF, 0x3F], + } + for value, expected_encoded in pairs.items(): + value &= 0xFFFFFFFF + encoded = UnsignedVarInt32.encode(value) + assert encoded == b"".join(struct.pack(">B", x) for x in expected_encoded) + assert value == UnsignedVarInt32.decode(io.BytesIO(encoded)) + + +def test_compact_data_structs(): + cs = CompactString() + encoded = cs.encode(None) + assert encoded == struct.pack("B", 0) + decoded = cs.decode(io.BytesIO(encoded)) + assert decoded is None + assert b"\x01" == cs.encode("") + assert "" == cs.decode(io.BytesIO(b"\x01")) + encoded = cs.encode("foobarbaz") + assert cs.decode(io.BytesIO(encoded)) == "foobarbaz" + + arr = CompactArray(CompactString()) + assert arr.encode(None) == b"\x00" + assert arr.decode(io.BytesIO(b"\x00")) is None + enc = arr.encode([]) + assert enc == b"\x01" + assert [] == arr.decode(io.BytesIO(enc)) + encoded = arr.encode(["foo", "bar", "baz", "quux"]) + assert arr.decode(io.BytesIO(encoded)) == ["foo", "bar", "baz", "quux"] + + enc = CompactBytes.encode(None) + assert enc == b"\x00" + assert CompactBytes.decode(io.BytesIO(b"\x00")) is None + enc = CompactBytes.encode(b"") + assert enc == b"\x01" + assert CompactBytes.decode(io.BytesIO(b"\x01")) == b"" + enc = CompactBytes.encode(b"foo") + assert CompactBytes.decode(io.BytesIO(enc)) == b"foo" + + +attr_names = [ + n for n in dir(Request) if isinstance(getattr(Request, n), abc.abstractproperty) +] + + +@pytest.mark.parametrize("klass", Request.__subclasses__()) +@pytest.mark.parametrize("attr_name", attr_names) +def test_request_type_conformance(klass, attr_name): + assert hasattr(klass, attr_name) + + +attr_names = [ + n for n in dir(Response) if isinstance(getattr(Response, n), abc.abstractproperty) +] + + +@pytest.mark.parametrize("klass", Response.__subclasses__()) +@pytest.mark.parametrize("attr_name", attr_names) +def test_response_type_conformance(klass, attr_name): + assert hasattr(klass, attr_name) diff --git a/tests/test_protocol_object_conversion.py b/tests/test_protocol_object_conversion.py new file mode 100644 index 00000000..5a5317cf --- /dev/null +++ b/tests/test_protocol_object_conversion.py @@ -0,0 +1,251 @@ +from aiokafka.protocol.admin import Request +from aiokafka.protocol.admin import Response +from aiokafka.protocol.types import Schema +from aiokafka.protocol.types import Array +from aiokafka.protocol.types import Int16 +from aiokafka.protocol.types import String + +import pytest + + +@pytest.mark.parametrize("superclass", (Request, Response)) +class TestObjectConversion: + def test_get_item(self, superclass): + class TestClass(superclass): + API_KEY = 0 + API_VERSION = 0 + RESPONSE_TYPE = None # To satisfy the Request ABC + SCHEMA = Schema(("myobject", Int16)) + + tc = TestClass(myobject=0) + assert tc.get_item("myobject") == 0 + with pytest.raises(KeyError): + tc.get_item("does-not-exist") + + def test_with_empty_schema(self, superclass): + class TestClass(superclass): + API_KEY = 0 + API_VERSION = 0 + RESPONSE_TYPE = None # To satisfy the Request ABC + SCHEMA = Schema() + + tc = TestClass() + tc.encode() + assert tc.to_object() == {} + + def test_with_basic_schema(self, superclass): + class TestClass(superclass): + API_KEY = 0 + API_VERSION = 0 + RESPONSE_TYPE = None # To satisfy the Request ABC + SCHEMA = Schema(("myobject", Int16)) + + tc = TestClass(myobject=0) + tc.encode() + assert tc.to_object() == {"myobject": 0} + + def test_with_basic_array_schema(self, superclass): + class TestClass(superclass): + API_KEY = 0 + API_VERSION = 0 + RESPONSE_TYPE = None # To satisfy the Request ABC + SCHEMA = Schema(("myarray", Array(Int16))) + + tc = TestClass(myarray=[1, 2, 3]) + tc.encode() + assert tc.to_object()["myarray"] == [1, 2, 3] + + def test_with_complex_array_schema(self, superclass): + class TestClass(superclass): + API_KEY = 0 + API_VERSION = 0 + RESPONSE_TYPE = None # To satisfy the Request ABC + SCHEMA = Schema( + ( + "myarray", + Array(("subobject", Int16), ("othersubobject", String("utf-8"))), + ) + ) + + tc = TestClass(myarray=[[10, "hello"]]) + tc.encode() + obj = tc.to_object() + assert len(obj["myarray"]) == 1 + assert obj["myarray"][0]["subobject"] == 10 + assert obj["myarray"][0]["othersubobject"] == "hello" + + def test_with_array_and_other(self, superclass): + class TestClass(superclass): + API_KEY = 0 + API_VERSION = 0 + RESPONSE_TYPE = None # To satisfy the Request ABC + SCHEMA = Schema( + ( + "myarray", + Array(("subobject", Int16), ("othersubobject", String("utf-8"))), + ), + ("notarray", Int16), + ) + + tc = TestClass(myarray=[[10, "hello"]], notarray=42) + + obj = tc.to_object() + assert len(obj["myarray"]) == 1 + assert obj["myarray"][0]["subobject"] == 10 + assert obj["myarray"][0]["othersubobject"] == "hello" + assert obj["notarray"] == 42 + + def test_with_nested_array(self, superclass): + class TestClass(superclass): + API_KEY = 0 + API_VERSION = 0 + RESPONSE_TYPE = None # To satisfy the Request ABC + SCHEMA = Schema( + ("myarray", Array(("subarray", Array(Int16)), ("otherobject", Int16))) + ) + + tc = TestClass( + myarray=[ + [[1, 2], 2], + [[2, 3], 4], + ] + ) + print(tc.encode()) + + obj = tc.to_object() + assert len(obj["myarray"]) == 2 + assert obj["myarray"][0]["subarray"] == [1, 2] + assert obj["myarray"][0]["otherobject"] == 2 + assert obj["myarray"][1]["subarray"] == [2, 3] + assert obj["myarray"][1]["otherobject"] == 4 + + def test_with_complex_nested_array(self, superclass): + class TestClass(superclass): + API_KEY = 0 + API_VERSION = 0 + RESPONSE_TYPE = None # To satisfy the Request ABC + SCHEMA = Schema( + ( + "myarray", + Array( + ( + "subarray", + Array( + ("innertest", String("utf-8")), + ("otherinnertest", String("utf-8")), + ), + ), + ("othersubarray", Array(Int16)), + ), + ), + ("notarray", String("utf-8")), + ) + + tc = TestClass( + myarray=[ + [[["hello", "hello"], ["hello again", "hello again"]], [0]], + [[["hello", "hello again"]], [1]], + ], + notarray="notarray", + ) + tc.encode() + + obj = tc.to_object() + + assert obj["notarray"] == "notarray" + myarray = obj["myarray"] + assert len(myarray) == 2 + + assert myarray[0]["othersubarray"] == [0] + assert len(myarray[0]["subarray"]) == 2 + assert myarray[0]["subarray"][0]["innertest"] == "hello" + assert myarray[0]["subarray"][0]["otherinnertest"] == "hello" + assert myarray[0]["subarray"][1]["innertest"] == "hello again" + assert myarray[0]["subarray"][1]["otherinnertest"] == "hello again" + + assert myarray[1]["othersubarray"] == [1] + assert len(myarray[1]["subarray"]) == 1 + assert myarray[1]["subarray"][0]["innertest"] == "hello" + assert myarray[1]["subarray"][0]["otherinnertest"] == "hello again" + + +def test_with_metadata_response(): + from aiokafka.protocol.metadata import MetadataResponse_v5 + + tc = MetadataResponse_v5( + throttle_time_ms=0, + brokers=[ + [0, "testhost0", 9092, "testrack0"], + [1, "testhost1", 9092, "testrack1"], + ], + cluster_id="abcd", + controller_id=0, + topics=[ + [ + 0, + "testtopic1", + False, + [ + [0, 0, 0, [0, 1], [0, 1], []], + [0, 1, 1, [1, 0], [1, 0], []], + ], + ], + [ + 0, + "other-test-topic", + True, + [ + [0, 0, 0, [0, 1], [0, 1], []], + ], + ], + ], + ) + tc.encode() # Make sure this object encodes successfully + + obj = tc.to_object() + + assert obj["throttle_time_ms"] == 0 + + assert len(obj["brokers"]) == 2 + assert obj["brokers"][0]["node_id"] == 0 + assert obj["brokers"][0]["host"] == "testhost0" + assert obj["brokers"][0]["port"] == 9092 + assert obj["brokers"][0]["rack"] == "testrack0" + assert obj["brokers"][1]["node_id"] == 1 + assert obj["brokers"][1]["host"] == "testhost1" + assert obj["brokers"][1]["port"] == 9092 + assert obj["brokers"][1]["rack"] == "testrack1" + + assert obj["cluster_id"] == "abcd" + assert obj["controller_id"] == 0 + + assert len(obj["topics"]) == 2 + assert obj["topics"][0]["error_code"] == 0 + assert obj["topics"][0]["topic"] == "testtopic1" + assert obj["topics"][0]["is_internal"] is False + assert len(obj["topics"][0]["partitions"]) == 2 + assert obj["topics"][0]["partitions"][0]["error_code"] == 0 + assert obj["topics"][0]["partitions"][0]["partition"] == 0 + assert obj["topics"][0]["partitions"][0]["leader"] == 0 + assert obj["topics"][0]["partitions"][0]["replicas"] == [0, 1] + assert obj["topics"][0]["partitions"][0]["isr"] == [0, 1] + assert obj["topics"][0]["partitions"][0]["offline_replicas"] == [] + assert obj["topics"][0]["partitions"][1]["error_code"] == 0 + assert obj["topics"][0]["partitions"][1]["partition"] == 1 + assert obj["topics"][0]["partitions"][1]["leader"] == 1 + assert obj["topics"][0]["partitions"][1]["replicas"] == [1, 0] + assert obj["topics"][0]["partitions"][1]["isr"] == [1, 0] + assert obj["topics"][0]["partitions"][1]["offline_replicas"] == [] + + assert obj["topics"][1]["error_code"] == 0 + assert obj["topics"][1]["topic"] == "other-test-topic" + assert obj["topics"][1]["is_internal"] is True + assert len(obj["topics"][1]["partitions"]) == 1 + assert obj["topics"][1]["partitions"][0]["error_code"] == 0 + assert obj["topics"][1]["partitions"][0]["partition"] == 0 + assert obj["topics"][1]["partitions"][0]["leader"] == 0 + assert obj["topics"][1]["partitions"][0]["replicas"] == [0, 1] + assert obj["topics"][1]["partitions"][0]["isr"] == [0, 1] + assert obj["topics"][1]["partitions"][0]["offline_replicas"] == [] + + tc.encode() diff --git a/tests/test_sender.py b/tests/test_sender.py index 01fdb1d8..099965ee 100644 --- a/tests/test_sender.py +++ b/tests/test_sender.py @@ -1,11 +1,17 @@ from unittest import mock -from ._testutil import ( - KafkaIntegrationTestCase, run_until_complete, kafka_versions +from aiokafka.client import AIOKafkaClient, CoordinationType, ConnectionGroup +from aiokafka.errors import ( + NoError, UnknownError, + CoordinatorNotAvailableError, NotCoordinatorError, + CoordinatorLoadInProgressError, ConcurrentTransactions, + UnknownTopicOrPartitionError, InvalidProducerEpoch, + ProducerFenced, InvalidProducerIdMapping, InvalidTxnState, + RequestTimedOutError, DuplicateSequenceNumber, KafkaError, + TopicAuthorizationFailedError, OperationNotAttempted, + TransactionalIdAuthorizationFailed, GroupAuthorizationFailedError ) - -from kafka.protocol.produce import ProduceRequest, ProduceResponse - +from aiokafka.producer.message_accumulator import MessageAccumulator from aiokafka.producer.sender import ( Sender, InitPIDHandler, AddPartitionsToTxnHandler, AddOffsetsToTxnHandler, TxnOffsetCommitHandler, EndTxnHandler, @@ -14,6 +20,8 @@ from aiokafka.producer.transaction_manager import ( TransactionManager, TransactionState ) +from aiokafka.protocol.metadata import MetadataRequest +from aiokafka.protocol.produce import ProduceRequest, ProduceResponse from aiokafka.protocol.transaction import ( InitProducerIdRequest, InitProducerIdResponse, AddPartitionsToTxnRequest, AddPartitionsToTxnResponse, @@ -21,25 +29,13 @@ TxnOffsetCommitRequest, TxnOffsetCommitResponse, EndTxnRequest, EndTxnResponse ) -from aiokafka.producer.message_accumulator import MessageAccumulator -from aiokafka.client import AIOKafkaClient, CoordinationType, ConnectionGroup from aiokafka.structs import TopicPartition, OffsetAndMetadata from aiokafka.util import get_running_loop -from aiokafka.errors import ( - NoError, UnknownError, - CoordinatorNotAvailableError, NotCoordinatorError, - CoordinatorLoadInProgressError, ConcurrentTransactions, - UnknownTopicOrPartitionError, InvalidProducerEpoch, - ProducerFenced, InvalidProducerIdMapping, InvalidTxnState, - RequestTimedOutError, DuplicateSequenceNumber, KafkaError, - TopicAuthorizationFailedError, OperationNotAttempted, - TransactionalIdAuthorizationFailed, GroupAuthorizationFailedError +from ._testutil import ( + KafkaIntegrationTestCase, run_until_complete, kafka_versions ) -from kafka.protocol.metadata import MetadataRequest - - LOG_APPEND_TIME = 1