diff --git a/kafka/protocol/new/__init__.py b/kafka/protocol/new/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/kafka/protocol/new/admin/__init__.py b/kafka/protocol/new/admin/__init__.py new file mode 100644 index 000000000..c87f521ec --- /dev/null +++ b/kafka/protocol/new/admin/__init__.py @@ -0,0 +1,27 @@ +from .acl import * +from .cluster import * +from .groups import * +from .topics import * + +__all__ = [ + 'CreateAclsRequest', 'CreateAclsResponse', + 'DeleteAclsRequest', 'DeleteAclsResponse', + 'DescribeAclsRequest', 'DescribeAclsResponse', + 'ACLResourceType', 'ACLOperation', + 'ACLPermissionType', 'ACLResourcePatternType', + + 'DescribeClusterRequest', 'DescribeClusterResponse', + 'DescribeConfigsRequest', 'DescribeConfigsResponse', + 'AlterConfigsRequest', 'AlterConfigsResponse', + 'DescribeLogDirsRequest', 'DescribeLogDirsResponse', + 'ElectLeadersRequest', 'ElectLeadersResponse', 'ElectionType', + + 'DescribeGroupsRequest', 'DescribeGroupsResponse', + 'ListGroupsRequest', 'ListGroupsResponse', + 'DeleteGroupsRequest', 'DeleteGroupsResponse', + + 'CreateTopicsRequest', 'CreateTopicsResponse', + 'DeleteTopicsRequest', 'DeleteTopicsResponse', + 'CreatePartitionsRequest', 'CreatePartitionsResponse', + 'DeleteRecordsRequest', 'DeleteRecordsResponse', +] diff --git a/kafka/protocol/new/admin/acl.py b/kafka/protocol/new/admin/acl.py new file mode 100644 index 000000000..a42999b05 --- /dev/null +++ b/kafka/protocol/new/admin/acl.py @@ -0,0 +1,83 @@ +from enum import IntEnum + +from ..api_message import ApiMessage + + +class CreateAclsRequest(ApiMessage): pass +class CreateAclsResponse(ApiMessage): pass + +class DeleteAclsRequest(ApiMessage): pass +class DeleteAclsResponse(ApiMessage): pass + +class DescribeAclsRequest(ApiMessage): pass +class DescribeAclsResponse(ApiMessage): pass + + +class ACLResourceType(IntEnum): + """Type of kafka resource to set ACL for + + The ANY value is only valid in a filter context + """ + UNKNOWN = 0, + ANY = 1, + CLUSTER = 4, + DELEGATION_TOKEN = 6, + GROUP = 3, + TOPIC = 2, + TRANSACTIONAL_ID = 5 + + +class ACLOperation(IntEnum): + """Type of operation + + The ANY value is only valid in a filter context + """ + UNKNOWN = 0, + ANY = 1, + ALL = 2, + READ = 3, + WRITE = 4, + CREATE = 5, + DELETE = 6, + ALTER = 7, + DESCRIBE = 8, + CLUSTER_ACTION = 9, + DESCRIBE_CONFIGS = 10, + ALTER_CONFIGS = 11, + IDEMPOTENT_WRITE = 12, + CREATE_TOKENS = 13, + DESCRIBE_TOKENS = 14 + + +class ACLPermissionType(IntEnum): + """An enumerated type of permissions + + The ANY value is only valid in a filter context + """ + UNKNOWN = 0, + ANY = 1, + DENY = 2, + ALLOW = 3 + + +class ACLResourcePatternType(IntEnum): + """An enumerated type of resource patterns + + More details on the pattern types and how they work + can be found in KIP-290 (Support for prefixed ACLs) + https://cwiki.apache.org/confluence/display/KAFKA/KIP-290%3A+Support+for+Prefixed+ACLs + """ + UNKNOWN = 0, + ANY = 1, + MATCH = 2, + LITERAL = 3, + PREFIXED = 4 + + +__all__ = [ + 'CreateAclsRequest', 'CreateAclsResponse', + 'DeleteAclsRequest', 'DeleteAclsResponse', + 'DescribeAclsRequest', 'DescribeAclsResponse', + 'ACLResourceType', 'ACLOperation', + 'ACLPermissionType', 'ACLResourcePatternType', +] diff --git a/kafka/protocol/new/admin/cluster.py b/kafka/protocol/new/admin/cluster.py new file mode 100644 index 000000000..fec7b4c91 --- /dev/null +++ b/kafka/protocol/new/admin/cluster.py @@ -0,0 +1,35 @@ +from enum import IntEnum + +from ..api_message import ApiMessage + + +class DescribeClusterRequest(ApiMessage): pass +class DescribeClusterResponse(ApiMessage): pass + +class DescribeConfigsRequest(ApiMessage): pass +class DescribeConfigsResponse(ApiMessage): pass + +class AlterConfigsRequest(ApiMessage): pass +class AlterConfigsResponse(ApiMessage): pass + +class DescribeLogDirsRequest(ApiMessage): pass +class DescribeLogDirsResponse(ApiMessage): pass + +class ElectLeadersRequest(ApiMessage): pass +class ElectLeadersResponse(ApiMessage): pass + +class ElectionType(IntEnum): + """ Leader election type + """ + + PREFERRED = 0, + UNCLEAN = 1 + + +__all__ = [ + 'DescribeClusterRequest', 'DescribeClusterResponse', + 'DescribeConfigsRequest', 'DescribeConfigsResponse', + 'AlterConfigsRequest', 'AlterConfigsResponse', + 'DescribeLogDirsRequest', 'DescribeLogDirsResponse', + 'ElectLeadersRequest', 'ElectLeadersResponse', 'ElectionType', +] diff --git a/kafka/protocol/new/admin/groups.py b/kafka/protocol/new/admin/groups.py new file mode 100644 index 000000000..cdb12d408 --- /dev/null +++ b/kafka/protocol/new/admin/groups.py @@ -0,0 +1,18 @@ +from ..api_message import ApiMessage + + +class DescribeGroupsRequest(ApiMessage): pass +class DescribeGroupsResponse(ApiMessage): pass + +class ListGroupsRequest(ApiMessage): pass +class ListGroupsResponse(ApiMessage): pass + +class DeleteGroupsRequest(ApiMessage): pass +class DeleteGroupsResponse(ApiMessage): pass + + +__all__ = [ + 'DescribeGroupsRequest', 'DescribeGroupsResponse', + 'ListGroupsRequest', 'ListGroupsResponse', + 'DeleteGroupsRequest', 'DeleteGroupsResponse', +] diff --git a/kafka/protocol/new/admin/topics.py b/kafka/protocol/new/admin/topics.py new file mode 100644 index 000000000..0ec681746 --- /dev/null +++ b/kafka/protocol/new/admin/topics.py @@ -0,0 +1,22 @@ +from ..api_message import ApiMessage + + +class CreateTopicsRequest(ApiMessage): pass +class CreateTopicsResponse(ApiMessage): pass + +class DeleteTopicsRequest(ApiMessage): pass +class DeleteTopicsResponse(ApiMessage): pass + +class CreatePartitionsRequest(ApiMessage): pass +class CreatePartitionsResponse(ApiMessage): pass + +class DeleteRecordsRequest(ApiMessage): pass +class DeleteRecordsResponse(ApiMessage): pass + + +__all__ = [ + 'CreateTopicsRequest', 'CreateTopicsResponse', + 'DeleteTopicsRequest', 'DeleteTopicsResponse', + 'CreatePartitionsRequest', 'CreatePartitionsResponse', + 'DeleteRecordsRequest', 'DeleteRecordsResponse', +] diff --git a/kafka/protocol/new/api_header.py b/kafka/protocol/new/api_header.py new file mode 100644 index 000000000..4acbc8198 --- /dev/null +++ b/kafka/protocol/new/api_header.py @@ -0,0 +1,59 @@ +from .data_container import DataContainer, SlotsBuilder +from .schemas import BaseField, StructField, load_json + + +class ApiHeaderMeta(SlotsBuilder): + def __new__(metacls, name, bases, attrs, **kw): + if kw.get('init', True): + json = load_json(name) + attrs['_json'] = json + attrs['_struct'] = StructField(json) + return super().__new__(metacls, name, bases, attrs, **kw) + + +class ApiHeader(DataContainer, metaclass=ApiHeaderMeta, init=False): + __slots__ = () + + def __init_subclass__(cls, **kw): + super().__init_subclass__(**kw) + if kw.get('init', True): + # pylint: disable=E1101 + assert cls._json['type'] == 'header' + cls._flexible_versions = BaseField.parse_versions(cls._json['flexibleVersions']) + cls._valid_versions = BaseField.parse_versions(cls._json['validVersions']) + + def encode(self, flexible=False): + # Request versions are 1-2, Response versions are 0-1 + version = self._flexible_versions[0] if flexible else self._valid_versions[0] # pylint: disable=E1136 + # compact=False is probably wrong, + # but it works to make sure that the client_id request header field + # is never encoded as compact (required to support ApiVersionsRequest for unsupported version) + return super().encode(version=version, compact=False, tagged=flexible) + + @classmethod + def decode(cls, data, flexible=False): + # Request versions are 1-2, Response versions are 0-1 + version = cls._flexible_versions[0] if flexible else cls._valid_versions[0] # pylint: disable=E1136 + return cls._struct.decode(data, version=version, compact=False, tagged=flexible, data_class=cls) + + +class ResponseClassRegistry: + _response_class_registry = {} + + @classmethod + def register_response_class(cls, response_class): + cls._response_class_registry[response_class.API_KEY] = response_class + + @classmethod + def get_response_class(cls, request_header): + response_class = cls._response_class_registry.get(request_header.request_api_key) + if response_class is not None: + return response_class[request_header.request_api_version] + + +class RequestHeader(ApiHeader): + def get_response_class(self): + return ResponseClassRegistry.get_response_class(self) + + +class ResponseHeader(ApiHeader): pass diff --git a/kafka/protocol/new/api_message.py b/kafka/protocol/new/api_message.py new file mode 100644 index 000000000..57324b4ff --- /dev/null +++ b/kafka/protocol/new/api_message.py @@ -0,0 +1,221 @@ +import io +import weakref + +from .api_header import RequestHeader, ResponseHeader, ResponseClassRegistry +from .data_container import DataContainer, SlotsBuilder +from .schemas import BaseField, StructField, load_json +from .schemas.fields.codecs import Int32 + +from kafka.util import classproperty + + +class VersionSubscriptable(type): + def __init__(cls, name, bases, attrs, **kw): + super().__init__(name, bases, attrs, **kw) + if kw.get('init', True): + # The primary message class has _version = None + # and a _VERSIONS dict that provides access to version-specific wrappers + # We also include cls[None] -> primary class to "exit" a version class + if getattr(cls, '_class_version', None) is None: + cls._class_version = None + cls._VERSIONS = {None: weakref.proxy(cls)} + + def __getitem__(cls, version): + # Use [] lookups to move from primary class to "versioned" classes + # which are simple wrappers around the primary class but with a _version attr + if cls._class_version is not None: + return cls._VERSIONS[None].__getitem__(version) + klass_name = cls.__name__ + '_v' + str(version) + if klass_name in cls._VERSIONS: + return cls._VERSIONS[klass_name] + cls._VERSIONS[klass_name] = type(klass_name, tuple(cls.mro()), {'_class_version': version}, init=False) + return cls._VERSIONS[klass_name] + + +class ApiMessageMeta(VersionSubscriptable, SlotsBuilder): + def __new__(metacls, name, bases, attrs, **kw): + # Pass init=False from base classes + if kw.get('init', True): + json = load_json(name) + if 'json_patch' in attrs: + json = attrs['json_patch'].__func__(metacls, json) + attrs['_json'] = json + attrs['_struct'] = StructField(json) + attrs['__doc__'] = json.get('doc') + attrs['__license__'] = json.get('license') + return super().__new__(metacls, name, bases, attrs, **kw) + + def __init__(cls, name, bases, attrs, **kw): + super().__init__(name, bases, attrs, **kw) + if kw.get('init', True): + # Ignore min valid version on request/response schemas + # We'll get the brokers supported versions via ApiVersionsRequest + if cls._struct._versions[0] > 0: + cls._struct._versions = (0, cls._struct._versions[1]) + # Configure the StructField to use our ApiMessage wrapper + # and not construct a default DataContainer + cls._struct.set_data_class(weakref.proxy(cls)) + + +class ApiMessage(DataContainer, metaclass=ApiMessageMeta, init=False): + __slots__ = ('_header', '_version') + + def __init_subclass__(cls, **kw): + super().__init_subclass__(**kw) + if kw.get('init', True): + # pylint: disable=E1101 + assert cls._json is not None + assert cls._json['type'] in ('request', 'response') + cls._flexible_versions = BaseField.parse_versions(cls._json['flexibleVersions']) + cls._valid_versions = BaseField.parse_versions(cls._json['validVersions']) + if not cls.is_request(): + ResponseClassRegistry.register_response_class(weakref.proxy(cls)) + + def __init__(self, *args, **kwargs): + super().__init__(*args, **kwargs) + self._header = None + self._version = None + if 'version' in kwargs: + self.API_VERSION = kwargs['version'] + + @classproperty + def name(cls): # pylint: disable=E0213 + return cls._json['name'] # pylint: disable=E1101 + + @classproperty + def type(cls): # pylint: disable=E0213 + return cls._json['type'] # pylint: disable=E1101 + + @classproperty + def API_KEY(cls): # pylint: disable=E0213 + return cls._json['apiKey'] # pylint: disable=E1101 + + @classproperty + def json(cls): # pylint: disable=E0213 + return cls._json # pylint: disable=E1101 + + @classproperty + def valid_versions(cls): # pylint: disable=E0213 + return cls._valid_versions + + @classproperty + def min_version(cls): # pylint: disable=E0213 + return 0 + + @classproperty + def max_version(cls): # pylint: disable=E0213 + if cls._valid_versions is not None: + return cls._valid_versions[1] # pylint: disable=E1136 + return None + + @classmethod + def flexible_version_q(cls, version): + if cls._flexible_versions is not None: + if cls._flexible_versions[0] <= version <= cls._flexible_versions[1]: # pylint: disable=E1136 + return True + return False + + @classmethod + def is_request(cls): + return cls.type == 'request' + + # allow override by api-specific classes (e.g., ProduceRequest) + def expect_response(self): + return True + + @property + def API_VERSION(self): + return self._version if self._version is not None else self._class_version # pylint: disable=E1101 + + @API_VERSION.setter + def API_VERSION(self, version): + if self._class_version is not None and self._class_version != version: # pylint: disable=E1101 + raise ValueError("Version has already been set by class") + if not 0 <= version <= self.max_version: + raise ValueError('Invalid version %s (max version is %s).' % (version, self.max_version)) + self._version = version + if self._header is not None: + self._header.request_api_version = version + + @property + def header(self): + return self._header + + @classproperty + def header_class(cls): # pylint: disable=E0213 + if cls.type == 'response': + return ResponseHeader + elif cls.type == 'request': + return RequestHeader + elif cls.type is None: + return None + else: + raise ValueError('Expected request or response type: %s' % cls.type) + + def with_header(self, correlation_id=0, client_id='kafka-python'): + if self.is_request(): + kwargs = { + 'request_api_key': self.API_KEY, + 'request_api_version': self.API_VERSION, + 'correlation_id': correlation_id, + 'client_id': client_id, + } + else: + kwargs = { + 'correlation_id': correlation_id, + } + self._header = self.header_class(**kwargs) + + # allow override by api-specific classes (e.g., ApiVersionsResponse) + def encode_header(self, flexible=False): + return self._header.encode(flexible=flexible) # pylint: disable=E1120 + + @classmethod + def parse_header(cls, data, flexible=False): + return cls.header_class.decode(data, flexible=flexible) # pylint: disable=E1101 + + def encode(self, version=None, header=False, framed=False): + if version is not None: + self.API_VERSION = version + if self.API_VERSION is None: + raise ValueError('Version required to encode data') + if header and self._header is None: + raise ValueError('No header found') + + flexible = self.flexible_version_q(self.API_VERSION) + encoded = self._struct.encode(self, version=self.API_VERSION, compact=flexible, tagged=flexible) + if not header and not framed: + return encoded + bits = [encoded] + if header: + bits.insert(0, self.encode_header(flexible=flexible)) + if framed: + bits.insert(0, Int32.encode(sum(map(len, bits)))) + return b''.join(bits) + + @classmethod + def decode(cls, data, version=None, header=False, framed=False): + version = cls._class_version if version is None else version + if version is None: + raise ValueError('Version required to decode data') + elif not 0 <= version <= cls.max_version: + raise ValueError('Invalid version %s (max version is %s).' % (version, cls.max_version)) + # Return current class except: current class is versioned and diff version is requested + if cls._class_version is not None and cls._class_version != version: + data_class = cls[version] + else: + data_class = cls + + flexible = cls.flexible_version_q(version) + if isinstance(data, bytes): + data = io.BytesIO(data) + if framed: + size = Int32.decode(data) + if header: + hdr = cls.parse_header(data, flexible=flexible) + else: + hdr = None + ret = cls._struct.decode(data, version=version, compact=flexible, tagged=flexible, data_class=data_class) + if hdr is not None: + ret._header = hdr + return ret diff --git a/kafka/protocol/new/consumer/__init__.py b/kafka/protocol/new/consumer/__init__.py new file mode 100644 index 000000000..e533ce1ad --- /dev/null +++ b/kafka/protocol/new/consumer/__init__.py @@ -0,0 +1,14 @@ +from .fetch import * +from .group import * + + +__all__ = [ + 'FetchRequest', 'FetchResponse', + 'ListOffsetsRequest', 'ListOffsetsResponse', + 'JoinGroupRequest', 'JoinGroupResponse', + 'SyncGroupRequest', 'SyncGroupResponse', + 'LeaveGroupRequest', 'LeaveGroupResponse', + 'HeartbeatRequest', 'HeartbeatResponse', + 'OffsetFetchRequest', 'OffsetFetchResponse', + 'OffsetCommitRequest', 'OffsetCommitResponse', +] diff --git a/kafka/protocol/new/consumer/fetch.py b/kafka/protocol/new/consumer/fetch.py new file mode 100644 index 000000000..975e3b404 --- /dev/null +++ b/kafka/protocol/new/consumer/fetch.py @@ -0,0 +1,18 @@ +from ..api_message import ApiMessage + + +class FetchRequest(ApiMessage): pass +class FetchResponse(ApiMessage): pass + +class ListOffsetsRequest(ApiMessage): pass +class ListOffsetsResponse(ApiMessage): pass + +class OffsetForLeaderEpochRequest(ApiMessage): pass +class OffsetForLeaderEpochResponse(ApiMessage): pass + + +__all__ = [ + 'FetchRequest', 'FetchResponse', + 'ListOffsetsRequest', 'ListOffsetsResponse', + 'OffsetForLeaderEpochRequest', 'OffsetForLeaderEpochResponse', +] diff --git a/kafka/protocol/new/consumer/group.py b/kafka/protocol/new/consumer/group.py new file mode 100644 index 000000000..59df82dee --- /dev/null +++ b/kafka/protocol/new/consumer/group.py @@ -0,0 +1,30 @@ +from ..api_message import ApiMessage + + +class JoinGroupRequest(ApiMessage): pass +class JoinGroupResponse(ApiMessage): pass + +class SyncGroupRequest(ApiMessage): pass +class SyncGroupResponse(ApiMessage): pass + +class LeaveGroupRequest(ApiMessage): pass +class LeaveGroupResponse(ApiMessage): pass + +class HeartbeatRequest(ApiMessage): pass +class HeartbeatResponse(ApiMessage): pass + +class OffsetFetchRequest(ApiMessage): pass +class OffsetFetchResponse(ApiMessage): pass + +class OffsetCommitRequest(ApiMessage): pass +class OffsetCommitResponse(ApiMessage): pass + + +__all__ = [ + 'JoinGroupRequest', 'JoinGroupResponse', + 'SyncGroupRequest', 'SyncGroupResponse', + 'LeaveGroupRequest', 'LeaveGroupResponse', + 'HeartbeatRequest', 'HeartbeatResponse', + 'OffsetFetchRequest', 'OffsetFetchResponse', + 'OffsetCommitRequest', 'OffsetCommitResponse', +] diff --git a/kafka/protocol/new/data_container.py b/kafka/protocol/new/data_container.py new file mode 100644 index 000000000..e620bd744 --- /dev/null +++ b/kafka/protocol/new/data_container.py @@ -0,0 +1,85 @@ +from kafka.util import classproperty + + +class SlotsBuilder(type): + def __new__(metacls, name, bases, attrs, **kw): + if attrs.get('_struct') is not None: + attrs['__slots__'] = attrs.get('__slots__', ()) + tuple(attrs['_struct'].fields.keys()) + return super().__new__(metacls, name, bases, attrs) + + +class DataContainer(metaclass=SlotsBuilder): + __slots__ = ('tags', 'unknown_tags') + _struct = None + + def __init_subclass__(cls, **kwargs): + super().__init_subclass__(**kwargs) + # Generate field data_classes and set as class attrs (by field.type_str) + if cls._struct is not None: + for field in cls._struct.fields.values(): + if field.is_struct() or field.is_struct_array(): + if not field.has_data_class(): + field.set_data_class(type(field.type_str, (DataContainer,), {'_struct': field})) + setattr(cls, field.type_str, field.data_class) + + def __init__(self, **field_vals): + assert self._struct is not None + self.tags = None + self.unknown_tags = None + for field in self._struct._fields: + if field.name in field_vals and field.tag is not None: + if self.tags is None: + self.tags = set() + self.tags.add(field.name) + setattr(self, field.name, field_vals.pop(field.name, field.default)) + + for name in list(field_vals.keys()): + if name.startswith('_'): + if self.unknown_tags is None: + self.unknown_tags = {} + self.unknown_tags[name] = field_vals.pop(name) + + if field_vals: + raise ValueError('Unrecognized fields for type %s: %s' % (self._struct.name, field_vals)) + + def encode(self, *args, **kwargs): + """Add version= to kwargs, otherwise pass-through to _struct""" + return self._struct.encode(self, *args, **kwargs) + + @classmethod + def decode(cls, data, **kwargs): + """Add version= to kwargs, otherwise pass-through to _struct""" + return cls._struct.decode(data, **kwargs) + + @classproperty + def fields(cls): # pylint: disable=E0213 + return cls._struct.fields + + def __repr__(self): + key_vals = ['%s=%s' % (field.name, repr(getattr(self, field.name))) + for field in self._struct._fields] + return self.__class__.__name__ + '(' + ', '.join(key_vals) + ')' + + def __eq__(self, other): + # For backwards compatibility Data struct is equal to tuple with same field values + if isinstance(other, tuple): + # TODO: handle fields changes by version? + if len(other) < len(self._struct._fields): + return False + for i, field in enumerate(self._struct._fields): + if getattr(self, field.name) != other[i]: + return False + if len(other) == len(self._struct._fields): + return True + elif len(other) == len(self._struct._fields) + 1 and isinstance(other[-1], dict) and other[-1] == {}: + # TODO: Handle non-empty tag dicts... + return True + return False + if self.__class__ != other.__class__: + return False + if self._struct != other._struct: + return False + for field in self._struct._fields: + if getattr(self, field.name) != getattr(other, field.name): + return False + return True diff --git a/kafka/protocol/new/metadata/__init__.py b/kafka/protocol/new/metadata/__init__.py new file mode 100644 index 000000000..23ccb798f --- /dev/null +++ b/kafka/protocol/new/metadata/__init__.py @@ -0,0 +1,10 @@ +from .api_versions import * +from .find_coordinator import * +from .metadata import * + + +__all__ = [ + 'ApiVersionsRequest', 'ApiVersionsResponse', + 'FindCoordinatorRequest', 'FindCoordinatorResponse', + 'MetadataRequest', 'MetadataResponse', +] diff --git a/kafka/protocol/new/metadata/api_versions.py b/kafka/protocol/new/metadata/api_versions.py new file mode 100644 index 000000000..a005028e6 --- /dev/null +++ b/kafka/protocol/new/metadata/api_versions.py @@ -0,0 +1,41 @@ +import io + +from ..api_message import ApiMessage +from ..schemas.fields.codecs import Int16, Int32 + + +class ApiVersionsRequest(ApiMessage): pass +class ApiVersionsResponse(ApiMessage): + # ApiVersionsResponse header never uses flexible formats, even if body does + @classmethod + def parse_header(cls, data, flexible=False): + return super().parse_header(data, flexible=False) + + def encode_header(self, flexible=False): + return super().encode_header(flexible=False) + + # ApiVersionsResponse body always decodes as version 0 when error is present + @classmethod + def decode(cls, data, version=None, header=False, framed=False): + if isinstance(data, bytes): + data = io.BytesIO(data) + if framed: + size = Int32.decode(data) + if header: + hdr = cls.parse_header(data) + else: + hdr = None + curr = data.tell() + err = Int16.decode(data) + data.seek(curr) + if err != 0: + version = 0 + ret = super().decode(data, version=version, header=False, framed=False) + if hdr is not None: + ret._header = hdr + return ret + + +__all__ = [ + 'ApiVersionsRequest', 'ApiVersionsResponse', +] diff --git a/kafka/protocol/new/metadata/find_coordinator.py b/kafka/protocol/new/metadata/find_coordinator.py new file mode 100644 index 000000000..9df38724f --- /dev/null +++ b/kafka/protocol/new/metadata/find_coordinator.py @@ -0,0 +1,10 @@ +from ..api_message import ApiMessage + + +class FindCoordinatorRequest(ApiMessage): pass +class FindCoordinatorResponse(ApiMessage): pass + + +__all__ = [ + 'FindCoordinatorRequest', 'FindCoordinatorResponse', +] diff --git a/kafka/protocol/new/metadata/metadata.py b/kafka/protocol/new/metadata/metadata.py new file mode 100644 index 000000000..38deb4b71 --- /dev/null +++ b/kafka/protocol/new/metadata/metadata.py @@ -0,0 +1,19 @@ +from ..api_message import ApiMessage + + +class MetadataRequest(ApiMessage): pass +class MetadataResponse(ApiMessage): + @classmethod + def json_patch(cls, json): + # cluster_authorized_operations + json['fields'][5]['name'] = 'authorized_operations' + json['fields'][5]['type'] = 'bitfield' + # topic_authorized_operations + json['fields'][4]['fields'][5]['name'] = 'authorized_operations' + json['fields'][4]['fields'][5]['type'] = 'bitfield' + return json + + +__all__ = [ + 'MetadataRequest', 'MetadataResponse', +] diff --git a/kafka/protocol/new/producer/__init__.py b/kafka/protocol/new/producer/__init__.py new file mode 100644 index 000000000..62e94637b --- /dev/null +++ b/kafka/protocol/new/producer/__init__.py @@ -0,0 +1,12 @@ +from .produce import * +from .transaction import * + + +__all__ = [ + 'ProduceRequest', 'ProduceResponse', + 'InitProducerIdRequest', 'InitProducerIdResponse', + 'AddPartitionsToTxnRequest', 'AddPartitionsToTxnResponse', + 'AddOffsetsToTxnRequest', 'AddOffsetsToTxnResponse', + 'EndTxnRequest', 'EndTxnResponse', + 'TxnOffsetCommitRequest', 'TxnOffsetCommitResponse', +] diff --git a/kafka/protocol/new/producer/produce.py b/kafka/protocol/new/producer/produce.py new file mode 100644 index 000000000..2cbcb30c3 --- /dev/null +++ b/kafka/protocol/new/producer/produce.py @@ -0,0 +1,15 @@ +from ..api_message import ApiMessage + + +class ProduceRequest(ApiMessage): + def expect_response(self): + if self.acks == 0: # pylint: disable=no-member + return False + return True + +class ProduceResponse(ApiMessage): pass + + +__all__ = [ + 'ProduceRequest', 'ProduceResponse', +] diff --git a/kafka/protocol/new/producer/transaction.py b/kafka/protocol/new/producer/transaction.py new file mode 100644 index 000000000..79d02be9b --- /dev/null +++ b/kafka/protocol/new/producer/transaction.py @@ -0,0 +1,26 @@ +from ..api_message import ApiMessage + + +class InitProducerIdRequest(ApiMessage): pass +class InitProducerIdResponse(ApiMessage): pass + +class AddPartitionsToTxnRequest(ApiMessage): pass +class AddPartitionsToTxnResponse(ApiMessage): pass + +class AddOffsetsToTxnRequest(ApiMessage): pass +class AddOffsetsToTxnResponse(ApiMessage): pass + +class EndTxnRequest(ApiMessage): pass +class EndTxnResponse(ApiMessage): pass + +class TxnOffsetCommitRequest(ApiMessage): pass +class TxnOffsetCommitResponse(ApiMessage): pass + + +__all__ = [ + 'InitProducerIdRequest', 'InitProducerIdResponse', + 'AddPartitionsToTxnRequest', 'AddPartitionsToTxnResponse', + 'AddOffsetsToTxnRequest', 'AddOffsetsToTxnResponse', + 'EndTxnRequest', 'EndTxnResponse', + 'TxnOffsetCommitRequest', 'TxnOffsetCommitResponse', +] diff --git a/kafka/protocol/new/sasl.py b/kafka/protocol/new/sasl.py new file mode 100644 index 000000000..5d22cd4fa --- /dev/null +++ b/kafka/protocol/new/sasl.py @@ -0,0 +1,14 @@ +from .api_message import ApiMessage + + +class SaslHandshakeRequest(ApiMessage): pass +class SaslHandshakeResponse(ApiMessage): pass + +class SaslAuthenticateRequest(ApiMessage): pass +class SaslAuthenticateResponse(ApiMessage): pass + + +__all__ = [ + 'SaslHandshakeRequest', 'SaslHandshakeResponse', + 'SaslAuthenticateRequest', 'SaslAuthenticateResponse', +] diff --git a/kafka/protocol/new/schemas/__init__.py b/kafka/protocol/new/schemas/__init__.py new file mode 100644 index 000000000..77425040c --- /dev/null +++ b/kafka/protocol/new/schemas/__init__.py @@ -0,0 +1,9 @@ +from .fields import * +from .load_json import load_json + + +__all__ = [ + "BaseField", "SimpleField", "StructField", + "ArrayField", "StructArrayField", + "load_json", +] diff --git a/kafka/protocol/new/schemas/fields/__init__.py b/kafka/protocol/new/schemas/fields/__init__.py new file mode 100644 index 000000000..c25f48632 --- /dev/null +++ b/kafka/protocol/new/schemas/fields/__init__.py @@ -0,0 +1,7 @@ +from .base import BaseField +from .simple import SimpleField +from .struct import StructField +from .array import ArrayField +from .struct_array import StructArrayField + +__all__ = ["BaseField", "SimpleField", "StructField", "ArrayField", "StructArrayField"] diff --git a/kafka/protocol/new/schemas/fields/array.py b/kafka/protocol/new/schemas/fields/array.py new file mode 100644 index 000000000..21b4ec0ab --- /dev/null +++ b/kafka/protocol/new/schemas/fields/array.py @@ -0,0 +1,75 @@ +from .base import BaseField +from .simple import SimpleField +from .codecs import ( + UnsignedVarInt32, Int32, +) + + +class ArrayField(BaseField): + @classmethod + def parse_inner_type(cls, json): + if 'fields' in json: + return + type_str = cls.parse_array_type(json) + if type_str is not None: + inner_json = {**json, 'type': type_str} + return SimpleField.parse_json(inner_json) + + @classmethod + def parse_array_type(cls, json): + if json['type'].startswith('[]'): + type_str = json['type'][2:] + assert not type_str.startswith('[]'), 'Unexpected double-array type: %s' % json['type'] + return type_str + + @classmethod + def parse_json(cls, json): + inner_type = cls.parse_inner_type(json) + if inner_type is not None: + return cls(json, array_of=inner_type) + + def __init__(self, json, array_of=None): + if array_of is None: + array_of = self.parse_inner_type(json) + assert array_of is not None, 'json does not contain a (simple) Array!' + super().__init__(json) + self.array_of = array_of # SimpleField + + def is_array(self): + return True + + def _calculate_default(self, default): + if default == 'null': + return None + elif not default: + return [] + else: + raise ValueError('Invalid default for field %s. The only valid default is empty or null.' % self._name) + + def encode(self, items, version=None, compact=False, tagged=False): + assert version is not None, 'version is required to encode Field' + if not self.for_version_q(version): + return b'' + if compact: + size = UnsignedVarInt32.encode(len(items) + 1 if items is not None else 0) + else: + size = Int32.encode(len(items) if items is not None else -1) + if items is None: + return size + fields = [self.array_of.encode(item, version=version, compact=compact, tagged=tagged) + for item in items] + return b''.join([size] + fields) + + def decode(self, data, version=None, compact=False, tagged=False): + assert version is not None, 'version is required to decode Field' + if not self.for_version_q(version): + return None + if compact: + size = UnsignedVarInt32.decode(data) + size -= 1 + else: + size = Int32.decode(data) + if size == -1: + return None + return [self.array_of.decode(data, version=version, compact=compact, tagged=tagged) + for _ in range(size)] diff --git a/kafka/protocol/new/schemas/fields/base.py b/kafka/protocol/new/schemas/fields/base.py new file mode 100644 index 000000000..4295edc6f --- /dev/null +++ b/kafka/protocol/new/schemas/fields/base.py @@ -0,0 +1,149 @@ +import re + + +class BaseField: + FIELD_TYPES = [] + + def __init_subclass__(cls, **kw): + cls.FIELD_TYPES.append(cls) + + @classmethod + def parse_json_fields(cls, json): + return tuple(map(BaseField.parse_json, json.get('fields', []))) or None # Note: DFS Field construction + + @classmethod + def parse_json(cls, json): + if 'type' not in json: + raise ValueError('No type found in json') + type_str = json['type'] + for field_type in cls.FIELD_TYPES: + maybe_type = field_type.parse_json(json) + if maybe_type is not None: + return maybe_type + else: + raise ValueError('Unable to parse field type: %s' % type_str) + + @classmethod + def underscore_name(cls, name): + return re.sub(r'(? 0, 'Unexpected empty fields in json' + type_str = cls.parse_array_type(json) + if type_str is None: + return + inner_json = {**json, 'type': type_str} + return StructField.parse_json(inner_json) + + @classmethod + def parse_json(cls, json): + inner_type = cls.parse_inner_type(json) + if inner_type is not None: + return cls(json, array_of=inner_type) + + def __init__(self, json, array_of=None): + if array_of is None: + array_of = self.parse_inner_type(json) + assert array_of is not None, 'json does not contain a StructArray!' + super().__init__(json, array_of=array_of) + # map_key will be (idx, field) of the mapKey field if found + self.map_key = next(filter(lambda x: x[1]._json.get('mapKey'), enumerate(self._fields)), None) + + @property + def type_str(self): + return self._type_str[2:] + + def is_struct_array(self): + return True + + @property + def fields(self): + return self.array_of.fields + + def has_data_class(self): + return self.array_of.has_data_class() + + def set_data_class(self, data_class): + return self.array_of.set_data_class(data_class) + + @property + def data_class(self): + return self.array_of.data_class + + def __call__(self, *args, **kw): + return self.data_class(*args, **kw) # pylint: disable=E1102 diff --git a/kafka/protocol/new/schemas/load_json.py b/kafka/protocol/new/schemas/load_json.py new file mode 100644 index 000000000..a2ee99eb8 --- /dev/null +++ b/kafka/protocol/new/schemas/load_json.py @@ -0,0 +1,37 @@ +import importlib.resources +import json +import re + + +def load_json(msg_type): + COMMENTS_REGEX = r"(?m)((?:^\s*//.*\n?)+)" + # Raises FileNotFoundError if not found + msg_json = importlib.resources.read_text(__package__ + '.resources', msg_type + '.json') + data = json.loads(re.sub(COMMENTS_REGEX, '', msg_json)) + comments = re.findall(COMMENTS_REGEX, msg_json) + if comments: + data['license'] = comments[0] + if len(comments) > 1: + data['doc'] = comments[1] + common_structs = {s['name']: s['fields'] for s in data.get('commonStructs', [])} + if common_structs: + _resolve_common_structs(data.get('fields', []), common_structs) + return data + + +def _resolve_common_structs(fields, common_structs): + for field in fields: + field_type = field['type'] + struct_name = None + if field_type.startswith('[]'): + inner_type = field_type[2:] + if inner_type and inner_type[0].isupper(): + struct_name = inner_type + elif field_type and field_type[0].isupper(): + struct_name = field_type + + if struct_name and struct_name in common_structs and 'fields' not in field: + field['fields'] = common_structs[struct_name] + + if 'fields' in field: + _resolve_common_structs(field['fields'], common_structs) diff --git a/kafka/protocol/new/schemas/resources/AddOffsetsToTxnRequest.json b/kafka/protocol/new/schemas/resources/AddOffsetsToTxnRequest.json new file mode 100644 index 000000000..9bebc8366 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/AddOffsetsToTxnRequest.json @@ -0,0 +1,40 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 25, + "type": "request", + "listeners": ["broker"], + "name": "AddOffsetsToTxnRequest", + // Version 1 is the same as version 0. + // + // Version 2 adds the support for new error code PRODUCER_FENCED. + // + // Version 3 enables flexible versions. + // + // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-4", + "flexibleVersions": "3+", + "fields": [ + { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId", + "about": "The transactional id corresponding to the transaction."}, + { "name": "ProducerId", "type": "int64", "versions": "0+", "entityType": "producerId", + "about": "Current producer id in use by the transactional id." }, + { "name": "ProducerEpoch", "type": "int16", "versions": "0+", + "about": "Current epoch associated with the producer id." }, + { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", + "about": "The unique group identifier." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/AddOffsetsToTxnResponse.json b/kafka/protocol/new/schemas/resources/AddOffsetsToTxnResponse.json new file mode 100644 index 000000000..6a713fea1 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/AddOffsetsToTxnResponse.json @@ -0,0 +1,35 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 25, + "type": "response", + "name": "AddOffsetsToTxnResponse", + // Starting in version 1, on quota violation brokers send out responses before throttling. + // + // Version 2 adds the support for new error code PRODUCER_FENCED. + // + // Version 3 enables flexible versions. + // + // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-4", + "flexibleVersions": "3+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The response error code, or 0 if there was no error." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/AddPartitionsToTxnRequest.json b/kafka/protocol/new/schemas/resources/AddPartitionsToTxnRequest.json new file mode 100644 index 000000000..68a45cdd0 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/AddPartitionsToTxnRequest.json @@ -0,0 +1,65 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 24, + "type": "request", + "listeners": ["broker"], + "name": "AddPartitionsToTxnRequest", + // Version 1 is the same as version 0. + // + // Version 2 adds the support for new error code PRODUCER_FENCED. + // + // Version 3 enables flexible versions. + // + // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions. + // + // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + // Versions 3 and below will be exclusively used by clients and versions 4 and above will be used by brokers. + "latestVersionUnstable": false, + "validVersions": "0-5", + "flexibleVersions": "3+", + "fields": [ + { "name": "Transactions", "type": "[]AddPartitionsToTxnTransaction", "versions": "4+", + "about": "List of transactions to add partitions to.", "fields": [ + { "name": "TransactionalId", "type": "string", "versions": "4+", "mapKey": true, "entityType": "transactionalId", + "about": "The transactional id corresponding to the transaction." }, + { "name": "ProducerId", "type": "int64", "versions": "4+", "entityType": "producerId", + "about": "Current producer id in use by the transactional id." }, + { "name": "ProducerEpoch", "type": "int16", "versions": "4+", + "about": "Current epoch associated with the producer id." }, + { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false, + "about": "Boolean to signify if we want to check if the partition is in the transaction rather than add it." }, + { "name": "Topics", "type": "[]AddPartitionsToTxnTopic", "versions": "4+", + "about": "The partitions to add to the transaction." } + ]}, + { "name": "V3AndBelowTransactionalId", "type": "string", "versions": "0-3", "entityType": "transactionalId", + "about": "The transactional id corresponding to the transaction." }, + { "name": "V3AndBelowProducerId", "type": "int64", "versions": "0-3", "entityType": "producerId", + "about": "Current producer id in use by the transactional id." }, + { "name": "V3AndBelowProducerEpoch", "type": "int16", "versions": "0-3", + "about": "Current epoch associated with the producer id." }, + { "name": "V3AndBelowTopics", "type": "[]AddPartitionsToTxnTopic", "versions": "0-3", + "about": "The partitions to add to the transaction." } + ], + "commonStructs": [ + { "name": "AddPartitionsToTxnTopic", "versions": "0+", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, "entityType": "topicName", + "about": "The name of the topic." }, + { "name": "Partitions", "type": "[]int32", "versions": "0+", + "about": "The partition indexes to add to the transaction." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/AddPartitionsToTxnResponse.json b/kafka/protocol/new/schemas/resources/AddPartitionsToTxnResponse.json new file mode 100644 index 000000000..a621740de --- /dev/null +++ b/kafka/protocol/new/schemas/resources/AddPartitionsToTxnResponse.json @@ -0,0 +1,60 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 24, + "type": "response", + "name": "AddPartitionsToTxnResponse", + // Starting in version 1, on quota violation brokers send out responses before throttling. + // + // Version 2 adds the support for new error code PRODUCER_FENCED. + // + // Version 3 enables flexible versions. + // + // Version 4 adds support to batch multiple transactions and a top level error code. + // + // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-5", + "flexibleVersions": "3+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "4+", "ignorable": true, + "about": "The response top level error code." }, + { "name": "ResultsByTransaction", "type": "[]AddPartitionsToTxnResult", "versions": "4+", + "about": "Results categorized by transactional ID.", "fields": [ + { "name": "TransactionalId", "type": "string", "versions": "4+", "mapKey": true, "entityType": "transactionalId", + "about": "The transactional id corresponding to the transaction." }, + { "name": "TopicResults", "type": "[]AddPartitionsToTxnTopicResult", "versions": "4+", + "about": "The results for each topic." } + ]}, + { "name": "ResultsByTopicV3AndBelow", "type": "[]AddPartitionsToTxnTopicResult", "versions": "0-3", + "about": "The results for each topic." } + ], + "commonStructs": [ + { "name": "AddPartitionsToTxnTopicResult", "versions": "0+", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, "entityType": "topicName", + "about": "The topic name." }, + { "name": "ResultsByPartition", "type": "[]AddPartitionsToTxnPartitionResult", "versions": "0+", + "about": "The results for each partition." } + ]}, + { "name": "AddPartitionsToTxnPartitionResult", "versions": "0+", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", "mapKey": true, + "about": "The partition indexes." }, + { "name": "PartitionErrorCode", "type": "int16", "versions": "0+", + "about": "The response error code." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/AlterConfigsRequest.json b/kafka/protocol/new/schemas/resources/AlterConfigsRequest.json new file mode 100644 index 000000000..b87091f2e --- /dev/null +++ b/kafka/protocol/new/schemas/resources/AlterConfigsRequest.json @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 33, + "type": "request", + "listeners": ["broker", "controller"], + "name": "AlterConfigsRequest", + // Version 1 is the same as version 0. + // Version 2 enables flexible versions. + "validVersions": "0-2", + "flexibleVersions": "2+", + "fields": [ + { "name": "Resources", "type": "[]AlterConfigsResource", "versions": "0+", + "about": "The updates for each resource.", "fields": [ + { "name": "ResourceType", "type": "int8", "versions": "0+", "mapKey": true, + "about": "The resource type." }, + { "name": "ResourceName", "type": "string", "versions": "0+", "mapKey": true, + "about": "The resource name." }, + { "name": "Configs", "type": "[]AlterableConfig", "versions": "0+", + "about": "The configurations.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, + "about": "The configuration key name." }, + { "name": "Value", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The value to set for the configuration key."} + ]} + ]}, + { "name": "ValidateOnly", "type": "bool", "versions": "0+", + "about": "True if we should validate the request, but not change the configurations."} + ] +} diff --git a/kafka/protocol/new/schemas/resources/AlterConfigsResponse.json b/kafka/protocol/new/schemas/resources/AlterConfigsResponse.json new file mode 100644 index 000000000..fab102b6c --- /dev/null +++ b/kafka/protocol/new/schemas/resources/AlterConfigsResponse.json @@ -0,0 +1,39 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 33, + "type": "response", + "name": "AlterConfigsResponse", + // Starting in version 1, on quota violation brokers send out responses before throttling. + // Version 2 enables flexible versions. + "validVersions": "0-2", + "flexibleVersions": "2+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Responses", "type": "[]AlterConfigsResourceResponse", "versions": "0+", + "about": "The responses for each resource.", "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The resource error code." }, + { "name": "ErrorMessage", "type": "string", "nullableVersions": "0+", "versions": "0+", + "about": "The resource error message, or null if there was no error." }, + { "name": "ResourceType", "type": "int8", "versions": "0+", + "about": "The resource type." }, + { "name": "ResourceName", "type": "string", "versions": "0+", + "about": "The resource name." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/ApiVersionsRequest.json b/kafka/protocol/new/schemas/resources/ApiVersionsRequest.json new file mode 100644 index 000000000..56170c966 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/ApiVersionsRequest.json @@ -0,0 +1,34 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 18, + "type": "request", + "listeners": ["broker", "controller"], + "name": "ApiVersionsRequest", + // Versions 0 through 2 of ApiVersionsRequest are the same. + // + // Version 3 is the first flexible version and adds ClientSoftwareName and ClientSoftwareVersion. + // + // Version 4 fixes KAFKA-17011, which blocked SupportedFeatures.MinVersion in the response from being 0. + "validVersions": "0-4", + "flexibleVersions": "3+", + "fields": [ + { "name": "ClientSoftwareName", "type": "string", "versions": "3+", + "ignorable": true, "about": "The name of the client." }, + { "name": "ClientSoftwareVersion", "type": "string", "versions": "3+", + "ignorable": true, "about": "The version of the client." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/ApiVersionsResponse.json b/kafka/protocol/new/schemas/resources/ApiVersionsResponse.json new file mode 100644 index 000000000..1017f2443 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/ApiVersionsResponse.json @@ -0,0 +1,79 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 18, + "type": "response", + "name": "ApiVersionsResponse", + // Version 1 adds throttle time to the response. + // + // Starting in version 2, on quota violation, brokers send out responses before throttling. + // + // Version 3 is the first flexible version. Tagged fields are only supported in the body but + // not in the header. The length of the header must not change in order to guarantee the + // backward compatibility. + // + // Starting from Apache Kafka 2.4 (KIP-511), ApiKeys field is populated with the supported + // versions of the ApiVersionsRequest when an UNSUPPORTED_VERSION error is returned. + // + // Version 4 fixes KAFKA-17011, which blocked SupportedFeatures.MinVersion from being 0. + "validVersions": "0-4", + "flexibleVersions": "3+", + "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The top-level error code." }, + { "name": "ApiKeys", "type": "[]ApiVersion", "versions": "0+", + "about": "The APIs supported by the broker.", "fields": [ + { "name": "ApiKey", "type": "int16", "versions": "0+", "mapKey": true, + "about": "The API index." }, + { "name": "MinVersion", "type": "int16", "versions": "0+", + "about": "The minimum supported version, inclusive." }, + { "name": "MaxVersion", "type": "int16", "versions": "0+", + "about": "The maximum supported version, inclusive." } + ]}, + { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "SupportedFeatures", "type": "[]SupportedFeatureKey", "ignorable": true, + "versions": "3+", "tag": 0, "taggedVersions": "3+", + "about": "Features supported by the broker. Note: in v0-v3, features with MinSupportedVersion = 0 are omitted.", + "fields": [ + { "name": "Name", "type": "string", "versions": "3+", "mapKey": true, + "about": "The name of the feature." }, + { "name": "MinVersion", "type": "int16", "versions": "3+", + "about": "The minimum supported version for the feature." }, + { "name": "MaxVersion", "type": "int16", "versions": "3+", + "about": "The maximum supported version for the feature." } + ] + }, + { "name": "FinalizedFeaturesEpoch", "type": "int64", "versions": "3+", + "tag": 1, "taggedVersions": "3+", "default": "-1", "ignorable": true, + "about": "The monotonically increasing epoch for the finalized features information. Valid values are >= 0. A value of -1 is special and represents unknown epoch." }, + { "name": "FinalizedFeatures", "type": "[]FinalizedFeatureKey", "ignorable": true, + "versions": "3+", "tag": 2, "taggedVersions": "3+", + "about": "List of cluster-wide finalized features. The information is valid only if FinalizedFeaturesEpoch >= 0.", + "fields": [ + { "name": "Name", "type": "string", "versions": "3+", "mapKey": true, + "about": "The name of the feature." }, + { "name": "MaxVersionLevel", "type": "int16", "versions": "3+", + "about": "The cluster-wide finalized max version level for the feature." }, + { "name": "MinVersionLevel", "type": "int16", "versions": "3+", + "about": "The cluster-wide finalized min version level for the feature." } + ] + }, + { "name": "ZkMigrationReady", "type": "bool", "versions": "3+", "taggedVersions": "3+", + "tag": 3, "ignorable": true, "default": "false", + "about": "Set by a KRaft controller if the required configurations for ZK migration are present." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/CreateAclsRequest.json b/kafka/protocol/new/schemas/resources/CreateAclsRequest.json new file mode 100644 index 000000000..d3a028b05 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/CreateAclsRequest.json @@ -0,0 +1,46 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 30, + "type": "request", + "listeners": ["broker", "controller"], + "name": "CreateAclsRequest", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // Version 1 adds resource pattern type. + // Version 2 enables flexible versions. + // Version 3 adds user resource type. + "validVersions": "1-3", + "flexibleVersions": "2+", + "fields": [ + { "name": "Creations", "type": "[]AclCreation", "versions": "0+", + "about": "The ACLs that we want to create.", "fields": [ + { "name": "ResourceType", "type": "int8", "versions": "0+", + "about": "The type of the resource." }, + { "name": "ResourceName", "type": "string", "versions": "0+", + "about": "The resource name for the ACL." }, + { "name": "ResourcePatternType", "type": "int8", "versions": "1+", "default": "3", + "about": "The pattern type for the ACL." }, + { "name": "Principal", "type": "string", "versions": "0+", + "about": "The principal for the ACL." }, + { "name": "Host", "type": "string", "versions": "0+", + "about": "The host for the ACL." }, + { "name": "Operation", "type": "int8", "versions": "0+", + "about": "The operation type for the ACL (read, write, etc.)." }, + { "name": "PermissionType", "type": "int8", "versions": "0+", + "about": "The permission type for the ACL (allow, deny, etc.)." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/CreateAclsResponse.json b/kafka/protocol/new/schemas/resources/CreateAclsResponse.json new file mode 100644 index 000000000..0e3168794 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/CreateAclsResponse.json @@ -0,0 +1,37 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 30, + "type": "response", + "name": "CreateAclsResponse", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // Starting in version 1, on quota violation, brokers send out responses before throttling. + // Version 2 enables flexible versions. + // Version 3 adds user resource type. + "validVersions": "1-3", + "flexibleVersions": "2+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Results", "type": "[]AclCreationResult", "versions": "0+", + "about": "The results for each ACL creation.", "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The result error, or zero if there was no error." }, + { "name": "ErrorMessage", "type": "string", "nullableVersions": "0+", "versions": "0+", + "about": "The result message, or null if there was no error." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/CreatePartitionsRequest.json b/kafka/protocol/new/schemas/resources/CreatePartitionsRequest.json new file mode 100644 index 000000000..95552a080 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/CreatePartitionsRequest.json @@ -0,0 +1,47 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 37, + "type": "request", + "listeners": ["broker", "controller"], + "name": "CreatePartitionsRequest", + // Version 1 is the same as version 0. + // + // Version 2 adds flexible version support + // + // Version 3 is identical to version 2 but may return a THROTTLING_QUOTA_EXCEEDED error + // in the response if the partitions creation is throttled (KIP-599). + "validVersions": "0-3", + "flexibleVersions": "2+", + "fields": [ + { "name": "Topics", "type": "[]CreatePartitionsTopic", "versions": "0+", + "about": "Each topic that we want to create new partitions inside.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, "entityType": "topicName", + "about": "The topic name." }, + { "name": "Count", "type": "int32", "versions": "0+", + "about": "The new partition count." }, + { "name": "Assignments", "type": "[]CreatePartitionsAssignment", "versions": "0+", "nullableVersions": "0+", + "about": "The new partition assignments.", "fields": [ + { "name": "BrokerIds", "type": "[]int32", "versions": "0+", "entityType": "brokerId", + "about": "The assigned broker IDs." } + ]} + ]}, + { "name": "TimeoutMs", "type": "int32", "versions": "0+", + "about": "The time in ms to wait for the partitions to be created." }, + { "name": "ValidateOnly", "type": "bool", "versions": "0+", + "about": "If true, then validate the request, but don't actually increase the number of partitions." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/CreatePartitionsResponse.json b/kafka/protocol/new/schemas/resources/CreatePartitionsResponse.json new file mode 100644 index 000000000..ef9f1f6f4 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/CreatePartitionsResponse.json @@ -0,0 +1,41 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 37, + "type": "response", + "name": "CreatePartitionsResponse", + // Starting in version 1, on quota violation, brokers send out responses before throttling. + // + // Version 2 adds flexible version support + // + // Version 3 is identical to version 2 but may return a THROTTLING_QUOTA_EXCEEDED error + // in the response if the partitions creation is throttled (KIP-599). + "validVersions": "0-3", + "flexibleVersions": "2+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Results", "type": "[]CreatePartitionsTopicResult", "versions": "0+", + "about": "The partition creation results for each topic.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The result error, or zero if there was no error."}, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", + "default": "null", "about": "The result message, or null if there was no error."} + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/CreateTopicsRequest.json b/kafka/protocol/new/schemas/resources/CreateTopicsRequest.json new file mode 100644 index 000000000..3ee039335 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/CreateTopicsRequest.json @@ -0,0 +1,65 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 19, + "type": "request", + "listeners": ["broker", "controller"], + "name": "CreateTopicsRequest", + // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline. + // + // Version 1 adds validateOnly. + // + // Version 4 makes partitions/replicationFactor optional even when assignments are not present (KIP-464) + // + // Version 5 is the first flexible version. + // Version 5 also returns topic configs in the response (KIP-525). + // + // Version 6 is identical to version 5 but may return a THROTTLING_QUOTA_EXCEEDED error + // in the response if the topics creation is throttled (KIP-599). + // + // Version 7 is the same as version 6. + "validVersions": "2-7", + "flexibleVersions": "5+", + "fields": [ + { "name": "Topics", "type": "[]CreatableTopic", "versions": "0+", + "about": "The topics to create.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, "entityType": "topicName", + "about": "The topic name." }, + { "name": "NumPartitions", "type": "int32", "versions": "0+", + "about": "The number of partitions to create in the topic, or -1 if we are either specifying a manual partition assignment or using the default partitions." }, + { "name": "ReplicationFactor", "type": "int16", "versions": "0+", + "about": "The number of replicas to create for each partition in the topic, or -1 if we are either specifying a manual partition assignment or using the default replication factor." }, + { "name": "Assignments", "type": "[]CreatableReplicaAssignment", "versions": "0+", + "about": "The manual partition assignment, or the empty array if we are using automatic assignment.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", "mapKey": true, + "about": "The partition index." }, + { "name": "BrokerIds", "type": "[]int32", "versions": "0+", "entityType": "brokerId", + "about": "The brokers to place the partition on." } + ]}, + { "name": "Configs", "type": "[]CreatableTopicConfig", "versions": "0+", + "about": "The custom topic configurations to set.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+" , "mapKey": true, + "about": "The configuration name." }, + { "name": "Value", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The configuration value." } + ]} + ]}, + { "name": "timeoutMs", "type": "int32", "versions": "0+", "default": "60000", + "about": "How long to wait in milliseconds before timing out the request." }, + { "name": "validateOnly", "type": "bool", "versions": "1+", "default": "false", "ignorable": false, + "about": "If true, check that the topics can be created as specified, but don't create anything." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/CreateTopicsResponse.json b/kafka/protocol/new/schemas/resources/CreateTopicsResponse.json new file mode 100644 index 000000000..94e728d08 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/CreateTopicsResponse.json @@ -0,0 +1,72 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 19, + "type": "response", + "name": "CreateTopicsResponse", + // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline. + // + // Version 1 adds a per-topic error message string. + // + // Version 2 adds the throttle time. + // + // Starting in version 3, on quota violation, brokers send out responses before throttling. + // + // Version 4 makes partitions/replicationFactor optional even when assignments are not present (KIP-464). + // + // Version 5 is the first flexible version. + // Version 5 also returns topic configs in the response (KIP-525). + // + // Version 6 is identical to version 5 but may return a THROTTLING_QUOTA_EXCEEDED error + // in the response if the topics creation is throttled (KIP-599). + // + // Version 7 returns the topic ID of the newly created topic if creation is successful. + "validVersions": "2-7", + "flexibleVersions": "5+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Topics", "type": "[]CreatableTopicResult", "versions": "0+", + "about": "Results for each topic we tried to create.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, "entityType": "topicName", + "about": "The topic name." }, + { "name": "TopicId", "type": "uuid", "versions": "7+", "ignorable": true, "about": "The unique topic ID."}, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." }, + { "name": "ErrorMessage", "type": "string", "versions": "1+", "nullableVersions": "0+", "ignorable": true, + "about": "The error message, or null if there was no error." }, + { "name": "TopicConfigErrorCode", "type": "int16", "versions": "5+", "tag": 0, "taggedVersions": "5+", "ignorable": true, + "about": "Optional topic config error returned if configs are not returned in the response." }, + { "name": "NumPartitions", "type": "int32", "versions": "5+", "default": "-1", "ignorable": true, + "about": "Number of partitions of the topic." }, + { "name": "ReplicationFactor", "type": "int16", "versions": "5+", "default": "-1", "ignorable": true, + "about": "Replication factor of the topic." }, + { "name": "Configs", "type": "[]CreatableTopicConfigs", "versions": "5+", "nullableVersions": "5+", "ignorable": true, + "about": "Configuration of the topic.", "fields": [ + { "name": "Name", "type": "string", "versions": "5+", + "about": "The configuration name." }, + { "name": "Value", "type": "string", "versions": "5+", "nullableVersions": "5+", + "about": "The configuration value." }, + { "name": "ReadOnly", "type": "bool", "versions": "5+", + "about": "True if the configuration is read-only." }, + { "name": "ConfigSource", "type": "int8", "versions": "5+", "default": "-1", "ignorable": true, + "about": "The configuration source." }, + { "name": "IsSensitive", "type": "bool", "versions": "5+", + "about": "True if this configuration is sensitive." } + ]} + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/DeleteAclsRequest.json b/kafka/protocol/new/schemas/resources/DeleteAclsRequest.json new file mode 100644 index 000000000..db605305a --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DeleteAclsRequest.json @@ -0,0 +1,46 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 31, + "type": "request", + "listeners": ["broker", "controller"], + "name": "DeleteAclsRequest", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // Version 1 adds the pattern type. + // Version 2 enables flexible versions. + // Version 3 adds the user resource type. + "validVersions": "1-3", + "flexibleVersions": "2+", + "fields": [ + { "name": "Filters", "type": "[]DeleteAclsFilter", "versions": "0+", + "about": "The filters to use when deleting ACLs.", "fields": [ + { "name": "ResourceTypeFilter", "type": "int8", "versions": "0+", + "about": "The resource type." }, + { "name": "ResourceNameFilter", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The resource name." }, + { "name": "PatternTypeFilter", "type": "int8", "versions": "1+", "default": "3", "ignorable": false, + "about": "The pattern type." }, + { "name": "PrincipalFilter", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The principal filter, or null to accept all principals." }, + { "name": "HostFilter", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The host filter, or null to accept all hosts." }, + { "name": "Operation", "type": "int8", "versions": "0+", + "about": "The ACL operation." }, + { "name": "PermissionType", "type": "int8", "versions": "0+", + "about": "The permission type." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/DeleteAclsResponse.json b/kafka/protocol/new/schemas/resources/DeleteAclsResponse.json new file mode 100644 index 000000000..516d589cb --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DeleteAclsResponse.json @@ -0,0 +1,59 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 31, + "type": "response", + "name": "DeleteAclsResponse", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // Version 1 adds the resource pattern type. + // Starting in version 1, on quota violation, brokers send out responses before throttling. + // Version 2 enables flexible versions. + // Version 3 adds the user resource type. + "validVersions": "1-3", + "flexibleVersions": "2+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "FilterResults", "type": "[]DeleteAclsFilterResult", "versions": "0+", + "about": "The results for each filter.", "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if the filter succeeded." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The error message, or null if the filter succeeded." }, + { "name": "MatchingAcls", "type": "[]DeleteAclsMatchingAcl", "versions": "0+", + "about": "The ACLs which matched this filter.", "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The deletion error code, or 0 if the deletion succeeded." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The deletion error message, or null if the deletion succeeded." }, + { "name": "ResourceType", "type": "int8", "versions": "0+", + "about": "The ACL resource type." }, + { "name": "ResourceName", "type": "string", "versions": "0+", + "about": "The ACL resource name." }, + { "name": "PatternType", "type": "int8", "versions": "1+", "default": "3", "ignorable": false, + "about": "The ACL resource pattern type." }, + { "name": "Principal", "type": "string", "versions": "0+", + "about": "The ACL principal." }, + { "name": "Host", "type": "string", "versions": "0+", + "about": "The ACL host." }, + { "name": "Operation", "type": "int8", "versions": "0+", + "about": "The ACL operation." }, + { "name": "PermissionType", "type": "int8", "versions": "0+", + "about": "The ACL permission type." } + ]} + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/DeleteGroupsRequest.json b/kafka/protocol/new/schemas/resources/DeleteGroupsRequest.json new file mode 100644 index 000000000..7d7c43717 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DeleteGroupsRequest.json @@ -0,0 +1,30 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 42, + "type": "request", + "listeners": ["broker"], + "name": "DeleteGroupsRequest", + // Version 1 is the same as version 0. + // + // Version 2 is the first flexible version. + "validVersions": "0-2", + "flexibleVersions": "2+", + "fields": [ + { "name": "GroupsNames", "type": "[]string", "versions": "0+", "entityType": "groupId", + "about": "The group names to delete." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/DeleteGroupsResponse.json b/kafka/protocol/new/schemas/resources/DeleteGroupsResponse.json new file mode 100644 index 000000000..168cde03b --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DeleteGroupsResponse.json @@ -0,0 +1,36 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 42, + "type": "response", + "name": "DeleteGroupsResponse", + // Starting in version 1, on quota violation, brokers send out responses before throttling. + // + // Version 2 is the first flexible version. + "validVersions": "0-2", + "flexibleVersions": "2+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Results", "type": "[]DeletableGroupResult", "versions": "0+", + "about": "The deletion results.", "fields": [ + { "name": "GroupId", "type": "string", "versions": "0+", "mapKey": true, "entityType": "groupId", + "about": "The group id." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The deletion error, or 0 if the deletion succeeded." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/DeleteRecordsRequest.json b/kafka/protocol/new/schemas/resources/DeleteRecordsRequest.json new file mode 100644 index 000000000..fc697944a --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DeleteRecordsRequest.json @@ -0,0 +1,42 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 21, + "type": "request", + "listeners": ["broker"], + "name": "DeleteRecordsRequest", + // Version 1 is the same as version 0. + + // Version 2 is the first flexible version. + "validVersions": "0-2", + "flexibleVersions": "2+", + "fields": [ + { "name": "Topics", "type": "[]DeleteRecordsTopic", "versions": "0+", + "about": "Each topic that we want to delete records from.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]DeleteRecordsPartition", "versions": "0+", + "about": "Each partition that we want to delete records from.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "Offset", "type": "int64", "versions": "0+", + "about": "The deletion offset." } + ]} + ]}, + { "name": "TimeoutMs", "type": "int32", "versions": "0+", + "about": "How long to wait for the deletion to complete, in milliseconds." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/DeleteRecordsResponse.json b/kafka/protocol/new/schemas/resources/DeleteRecordsResponse.json new file mode 100644 index 000000000..bfc0a5639 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DeleteRecordsResponse.json @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 21, + "type": "response", + "name": "DeleteRecordsResponse", + // Starting in version 1, on quota violation, brokers send out responses before throttling. + + // Version 2 is the first flexible version. + "validVersions": "0-2", + "flexibleVersions": "2+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Topics", "type": "[]DeleteRecordsTopicResult", "versions": "0+", + "about": "Each topic that we wanted to delete records from.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]DeleteRecordsPartitionResult", "versions": "0+", + "about": "Each partition that we wanted to delete records from.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", "mapKey": true, + "about": "The partition index." }, + { "name": "LowWatermark", "type": "int64", "versions": "0+", + "about": "The partition low water mark." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The deletion error code, or 0 if the deletion succeeded." } + ]} + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/DeleteTopicsRequest.json b/kafka/protocol/new/schemas/resources/DeleteTopicsRequest.json new file mode 100644 index 000000000..35c9eb28f --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DeleteTopicsRequest.json @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 20, + "type": "request", + "listeners": ["broker", "controller"], + "name": "DeleteTopicsRequest", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // Versions 0, 1, 2, and 3 are the same. + // + // Version 4 is the first flexible version. + // + // Version 5 adds ErrorMessage in the response and may return a THROTTLING_QUOTA_EXCEEDED error + // in the response if the topics deletion is throttled (KIP-599). + // + // Version 6 reorganizes topics, adds topic IDs and allows topic names to be null. + "validVersions": "1-6", + "flexibleVersions": "4+", + "fields": [ + { "name": "Topics", "type": "[]DeleteTopicState", "versions": "6+", "about": "The name or topic ID of the topic.", + "fields": [ + {"name": "Name", "type": "string", "versions": "6+", "nullableVersions": "6+", "default": "null", "entityType": "topicName", "about": "The topic name."}, + {"name": "TopicId", "type": "uuid", "versions": "6+", "about": "The unique topic ID."} + ]}, + { "name": "TopicNames", "type": "[]string", "versions": "0-5", "entityType": "topicName", "ignorable": true, + "about": "The names of the topics to delete." }, + { "name": "TimeoutMs", "type": "int32", "versions": "0+", + "about": "The length of time in milliseconds to wait for the deletions to complete." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/DeleteTopicsResponse.json b/kafka/protocol/new/schemas/resources/DeleteTopicsResponse.json new file mode 100644 index 000000000..a31d41c2b --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DeleteTopicsResponse.json @@ -0,0 +1,52 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 20, + "type": "response", + "name": "DeleteTopicsResponse", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // + // Version 1 adds the throttle time. + // + // Starting in version 2, on quota violation, brokers send out responses before throttling. + // + // Starting in version 3, a TOPIC_DELETION_DISABLED error code may be returned. + // + // Version 4 is the first flexible version. + // + // Version 5 adds ErrorMessage in the response and may return a THROTTLING_QUOTA_EXCEEDED error + // in the response if the topics deletion is throttled (KIP-599). + // + // Version 6 adds topic ID to responses. An UNSUPPORTED_VERSION error code will be returned when attempting to + // delete using topic IDs when IBP < 2.8. UNKNOWN_TOPIC_ID error code will be returned when IBP is at least 2.8, but + // the topic ID was not found. + "validVersions": "1-6", + "flexibleVersions": "4+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Responses", "type": "[]DeletableTopicResult", "versions": "0+", + "about": "The results for each topic we tried to delete.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "nullableVersions": "6+", "mapKey": true, "entityType": "topicName", + "about": "The topic name." }, + {"name": "TopicId", "type": "uuid", "versions": "6+", "ignorable": true, "about": "The unique topic ID."}, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The deletion error, or 0 if the deletion succeeded." }, + { "name": "ErrorMessage", "type": "string", "versions": "5+", "nullableVersions": "5+", "ignorable": true, "default": "null", + "about": "The error message, or null if there was no error." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/DescribeAclsRequest.json b/kafka/protocol/new/schemas/resources/DescribeAclsRequest.json new file mode 100644 index 000000000..23883c154 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DescribeAclsRequest.json @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 29, + "type": "request", + "listeners": ["broker", "controller"], + "name": "DescribeAclsRequest", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // Version 1 adds resource pattern type. + // Version 2 enables flexible versions. + // Version 3 adds user resource type. + "validVersions": "1-3", + "flexibleVersions": "2+", + "fields": [ + { "name": "ResourceTypeFilter", "type": "int8", "versions": "0+", + "about": "The resource type." }, + { "name": "ResourceNameFilter", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The resource name, or null to match any resource name." }, + { "name": "PatternTypeFilter", "type": "int8", "versions": "1+", "default": "3", "ignorable": false, + "about": "The resource pattern to match." }, + { "name": "PrincipalFilter", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The principal to match, or null to match any principal." }, + { "name": "HostFilter", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The host to match, or null to match any host." }, + { "name": "Operation", "type": "int8", "versions": "0+", + "about": "The operation to match." }, + { "name": "PermissionType", "type": "int8", "versions": "0+", + "about": "The permission type to match." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/DescribeAclsResponse.json b/kafka/protocol/new/schemas/resources/DescribeAclsResponse.json new file mode 100644 index 000000000..e11ce4658 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DescribeAclsResponse.json @@ -0,0 +1,55 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 29, + "type": "response", + "name": "DescribeAclsResponse", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // Version 1 adds PatternType. + // Starting in version 1, on quota violation, brokers send out responses before throttling. + // Version 2 enables flexible versions. + // Version 3 adds user resource type. + "validVersions": "1-3", + "flexibleVersions": "2+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The error message, or null if there was no error." }, + { "name": "Resources", "type": "[]DescribeAclsResource", "versions": "0+", + "about": "Each Resource that is referenced in an ACL.", "fields": [ + { "name": "ResourceType", "type": "int8", "versions": "0+", + "about": "The resource type." }, + { "name": "ResourceName", "type": "string", "versions": "0+", + "about": "The resource name." }, + { "name": "PatternType", "type": "int8", "versions": "1+", "default": "3", "ignorable": false, + "about": "The resource pattern type." }, + { "name": "Acls", "type": "[]AclDescription", "versions": "0+", + "about": "The ACLs.", "fields": [ + { "name": "Principal", "type": "string", "versions": "0+", + "about": "The ACL principal." }, + { "name": "Host", "type": "string", "versions": "0+", + "about": "The ACL host." }, + { "name": "Operation", "type": "int8", "versions": "0+", + "about": "The ACL operation." }, + { "name": "PermissionType", "type": "int8", "versions": "0+", + "about": "The ACL permission type." } + ]} + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/DescribeClusterRequest.json b/kafka/protocol/new/schemas/resources/DescribeClusterRequest.json new file mode 100644 index 000000000..9c17c6b1b --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DescribeClusterRequest.json @@ -0,0 +1,35 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 60, + "type": "request", + "listeners": ["broker", "controller"], + "name": "DescribeClusterRequest", + // + // Version 1 adds EndpointType for KIP-919 support. + // Version 2 adds IncludeFencedBrokers for KIP-1073 support. + // + "validVersions": "0-2", + "flexibleVersions": "0+", + "fields": [ + { "name": "IncludeClusterAuthorizedOperations", "type": "bool", "versions": "0+", + "about": "Whether to include cluster authorized operations." }, + { "name": "EndpointType", "type": "int8", "versions": "1+", "default": "1", + "about": "The endpoint type to describe. 1=brokers, 2=controllers." }, + { "name": "IncludeFencedBrokers", "type": "bool", "versions": "2+", + "about": "Whether to include fenced brokers when listing brokers." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/DescribeClusterResponse.json b/kafka/protocol/new/schemas/resources/DescribeClusterResponse.json new file mode 100644 index 000000000..a17e427c8 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DescribeClusterResponse.json @@ -0,0 +1,56 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 60, + "type": "response", + "name": "DescribeClusterResponse", + // + // Version 1 adds the EndpointType field, and makes MISMATCHED_ENDPOINT_TYPE and + // UNSUPPORTED_ENDPOINT_TYPE valid top-level response error codes. + // Version 2 adds IsFenced field to Brokers for KIP-1073 support. + // + "validVersions": "0-2", + "flexibleVersions": "0+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The top-level error code, or 0 if there was no error." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The top-level error message, or null if there was no error." }, + { "name": "EndpointType", "type": "int8", "versions": "1+", "default": "1", + "about": "The endpoint type that was described. 1=brokers, 2=controllers." }, + { "name": "ClusterId", "type": "string", "versions": "0+", + "about": "The cluster ID that responding broker belongs to." }, + { "name": "ControllerId", "type": "int32", "versions": "0+", "default": "-1", "entityType": "brokerId", + "about": "The ID of the controller broker." }, + { "name": "Brokers", "type": "[]DescribeClusterBroker", "versions": "0+", + "about": "Each broker in the response.", "fields": [ + { "name": "BrokerId", "type": "int32", "versions": "0+", "mapKey": true, "entityType": "brokerId", + "about": "The broker ID." }, + { "name": "Host", "type": "string", "versions": "0+", + "about": "The broker hostname." }, + { "name": "Port", "type": "int32", "versions": "0+", + "about": "The broker port." }, + { "name": "Rack", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + "about": "The rack of the broker, or null if it has not been assigned to a rack." }, + { "name": "IsFenced", "type": "bool", "versions": "2+", + "about": "Whether the broker is fenced" } + ]}, + { "name": "ClusterAuthorizedOperations", "type": "int32", "versions": "0+", "default": "-2147483648", + "about": "32-bit bitfield to represent authorized operations for this cluster." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/DescribeConfigsRequest.json b/kafka/protocol/new/schemas/resources/DescribeConfigsRequest.json new file mode 100644 index 000000000..d1a85a67f --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DescribeConfigsRequest.json @@ -0,0 +1,42 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 32, + "type": "request", + "listeners": ["broker", "controller"], + "name": "DescribeConfigsRequest", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // Version 1 adds IncludeSynonyms and removes IsDefault. + // Version 2 is the same as version 1. + // Version 4 enables flexible versions. + "validVersions": "1-4", + "flexibleVersions": "4+", + "fields": [ + { "name": "Resources", "type": "[]DescribeConfigsResource", "versions": "0+", + "about": "The resources whose configurations we want to describe.", "fields": [ + { "name": "ResourceType", "type": "int8", "versions": "0+", + "about": "The resource type." }, + { "name": "ResourceName", "type": "string", "versions": "0+", + "about": "The resource name." }, + { "name": "ConfigurationKeys", "type": "[]string", "versions": "0+", "nullableVersions": "0+", + "about": "The configuration keys to list, or null to list all configuration keys." } + ]}, + { "name": "IncludeSynonyms", "type": "bool", "versions": "1+", "default": "false", "ignorable": false, + "about": "True if we should include all synonyms." }, + { "name": "IncludeDocumentation", "type": "bool", "versions": "3+", "default": "false", "ignorable": false, + "about": "True if we should include configuration documentation." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/DescribeConfigsResponse.json b/kafka/protocol/new/schemas/resources/DescribeConfigsResponse.json new file mode 100644 index 000000000..b7127b69f --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DescribeConfigsResponse.json @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 32, + "type": "response", + "name": "DescribeConfigsResponse", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // Version 1 adds ConfigSource and the synonyms. + // Starting in version 2, on quota violation, brokers send out responses before throttling. + // Version 4 enables flexible versions. + "validVersions": "1-4", + "flexibleVersions": "4+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Results", "type": "[]DescribeConfigsResult", "versions": "0+", + "about": "The results for each resource.", "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if we were able to successfully describe the configurations." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The error message, or null if we were able to successfully describe the configurations." }, + { "name": "ResourceType", "type": "int8", "versions": "0+", + "about": "The resource type." }, + { "name": "ResourceName", "type": "string", "versions": "0+", + "about": "The resource name." }, + { "name": "Configs", "type": "[]DescribeConfigsResourceResult", "versions": "0+", + "about": "Each listed configuration.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", + "about": "The configuration name." }, + { "name": "Value", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The configuration value." }, + { "name": "ReadOnly", "type": "bool", "versions": "0+", + "about": "True if the configuration is read-only." }, + { "name": "ConfigSource", "type": "int8", "versions": "1+", "default": "-1", "ignorable": true, + "about": "The configuration source." }, + { "name": "IsSensitive", "type": "bool", "versions": "0+", + "about": "True if this configuration is sensitive." }, + { "name": "Synonyms", "type": "[]DescribeConfigsSynonym", "versions": "1+", "ignorable": true, + "about": "The synonyms for this configuration key.", "fields": [ + { "name": "Name", "type": "string", "versions": "1+", + "about": "The synonym name." }, + { "name": "Value", "type": "string", "versions": "1+", "nullableVersions": "0+", + "about": "The synonym value." }, + { "name": "Source", "type": "int8", "versions": "1+", + "about": "The synonym source." } + ]}, + { "name": "ConfigType", "type": "int8", "versions": "3+", "default": "0", "ignorable": true, + "about": "The configuration data type. Type can be one of the following values - BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS, PASSWORD." }, + { "name": "Documentation", "type": "string", "versions": "3+", "nullableVersions": "0+", "ignorable": true, + "about": "The configuration documentation." } + ]} + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/DescribeGroupsRequest.json b/kafka/protocol/new/schemas/resources/DescribeGroupsRequest.json new file mode 100644 index 000000000..cec56852c --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DescribeGroupsRequest.json @@ -0,0 +1,38 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 15, + "type": "request", + "listeners": ["broker"], + "name": "DescribeGroupsRequest", + // Versions 1 and 2 are the same as version 0. + // + // Starting in version 3, authorized operations can be requested. + // + // Starting in version 4, the response will include group.instance.id info for members. + // + // Version 5 is the first flexible version. + // + // Version 6 returns error code GROUP_ID_NOT_FOUND if the group ID is not found (KIP-1043). + "validVersions": "0-6", + "flexibleVersions": "5+", + "fields": [ + { "name": "Groups", "type": "[]string", "versions": "0+", "entityType": "groupId", + "about": "The names of the groups to describe." }, + { "name": "IncludeAuthorizedOperations", "type": "bool", "versions": "3+", + "about": "Whether to include authorized operations." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/DescribeGroupsResponse.json b/kafka/protocol/new/schemas/resources/DescribeGroupsResponse.json new file mode 100644 index 000000000..2101fd7ca --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DescribeGroupsResponse.json @@ -0,0 +1,74 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 15, + "type": "response", + "name": "DescribeGroupsResponse", + // Version 1 added throttle time. + // + // Starting in version 2, on quota violation, brokers send out responses before throttling. + // + // Starting in version 3, brokers can send authorized operations. + // + // Starting in version 4, the response will optionally include group.instance.id info for members. + // + // Version 5 is the first flexible version. + // + // Version 6 returns error code GROUP_ID_NOT_FOUND if the group ID is not found (KIP-1043). + "validVersions": "0-6", + "flexibleVersions": "5+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Groups", "type": "[]DescribedGroup", "versions": "0+", + "about": "Each described group.", "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The describe error, or 0 if there was no error." }, + { "name": "ErrorMessage", "type": "string", "versions": "6+", "nullableVersions": "6+", "default": "null", + "about": "The describe error message, or null if there was no error." }, + { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", + "about": "The group ID string." }, + { "name": "GroupState", "type": "string", "versions": "0+", + "about": "The group state string, or the empty string." }, + { "name": "ProtocolType", "type": "string", "versions": "0+", + "about": "The group protocol type, or the empty string." }, + // ProtocolData is currently only filled in if the group state is in the Stable state. + { "name": "ProtocolData", "type": "string", "versions": "0+", + "about": "The group protocol data, or the empty string." }, + // N.B. If the group is in the Dead state, the members array will always be empty. + { "name": "Members", "type": "[]DescribedGroupMember", "versions": "0+", + "about": "The group members.", "fields": [ + { "name": "MemberId", "type": "string", "versions": "0+", + "about": "The member id." }, + { "name": "GroupInstanceId", "type": "string", "versions": "4+", "ignorable": true, + "nullableVersions": "4+", "default": "null", + "about": "The unique identifier of the consumer instance provided by end user." }, + { "name": "ClientId", "type": "string", "versions": "0+", + "about": "The client ID used in the member's latest join group request." }, + { "name": "ClientHost", "type": "string", "versions": "0+", + "about": "The client host." }, + // This is currently only provided if the group is in the Stable state. + { "name": "MemberMetadata", "type": "bytes", "versions": "0+", + "about": "The metadata corresponding to the current group protocol in use." }, + // This is currently only provided if the group is in the Stable state. + { "name": "MemberAssignment", "type": "bytes", "versions": "0+", + "about": "The current assignment provided by the group leader." } + ]}, + { "name": "AuthorizedOperations", "type": "int32", "versions": "3+", "default": "-2147483648", + "about": "32-bit bitfield to represent authorized operations for this group." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/DescribeLogDirsRequest.json b/kafka/protocol/new/schemas/resources/DescribeLogDirsRequest.json new file mode 100644 index 000000000..4f3bfa2c5 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DescribeLogDirsRequest.json @@ -0,0 +1,37 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 35, + "type": "request", + "listeners": ["broker"], + "name": "DescribeLogDirsRequest", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // Version 1 is the same as version 0. + // Version 2 is the first flexible version. + // Version 3 is the same as version 2 (new field in response). + // Version 4 is the same as version 2 (new fields in response). + "validVersions": "1-4", + "flexibleVersions": "2+", + "fields": [ + { "name": "Topics", "type": "[]DescribableLogDirTopic", "versions": "0+", "nullableVersions": "0+", + "about": "Each topic that we want to describe log directories for, or null for all topics.", "fields": [ + { "name": "Topic", "type": "string", "versions": "0+", "entityType": "topicName", "mapKey": true, + "about": "The topic name." }, + { "name": "Partitions", "type": "[]int32", "versions": "0+", + "about": "The partition indexes." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/DescribeLogDirsResponse.json b/kafka/protocol/new/schemas/resources/DescribeLogDirsResponse.json new file mode 100644 index 000000000..d05785fe8 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/DescribeLogDirsResponse.json @@ -0,0 +1,61 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 35, + "type": "response", + "name": "DescribeLogDirsResponse", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // Starting in version 1, on quota violation, brokers send out responses before throttling. + // Version 2 is the first flexible version. + // Version 3 adds the top-level ErrorCode field + // Version 4 adds the TotalBytes and UsableBytes fields + "validVersions": "1-4", + "flexibleVersions": "2+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "3+", + "ignorable": true, "about": "The error code, or 0 if there was no error." }, + { "name": "Results", "type": "[]DescribeLogDirsResult", "versions": "0+", + "about": "The log directories.", "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." }, + { "name": "LogDir", "type": "string", "versions": "0+", + "about": "The absolute log directory path." }, + { "name": "Topics", "type": "[]DescribeLogDirsTopic", "versions": "0+", + "about": "The topics.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]DescribeLogDirsPartition", "versions": "0+", + "about": "The partitions.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "PartitionSize", "type": "int64", "versions": "0+", + "about": "The size of the log segments in this partition in bytes." }, + { "name": "OffsetLag", "type": "int64", "versions": "0+", + "about": "The lag of the log's LEO w.r.t. partition's HW (if it is the current log for the partition) or current replica's LEO (if it is the future log for the partition)." }, + { "name": "IsFutureKey", "type": "bool", "versions": "0+", + "about": "True if this log is created by AlterReplicaLogDirsRequest and will replace the current log of the replica in the future." }]} + ]}, + { "name": "TotalBytes", "type": "int64", "versions": "4+", "ignorable": true, "default": "-1", + "about": "The total size in bytes of the volume the log directory is in." + }, + { "name": "UsableBytes", "type": "int64", "versions": "4+", "ignorable": true, "default": "-1", + "about": "The usable size in bytes of the volume the log directory is in." + } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/ElectLeadersRequest.json b/kafka/protocol/new/schemas/resources/ElectLeadersRequest.json new file mode 100644 index 000000000..bce04585a --- /dev/null +++ b/kafka/protocol/new/schemas/resources/ElectLeadersRequest.json @@ -0,0 +1,41 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 43, + "type": "request", + "listeners": ["broker", "controller"], + "name": "ElectLeadersRequest", + // Version 1 implements multiple leader election types, as described by KIP-460. + // + // Version 2 is the first flexible version. + "validVersions": "0-2", + "flexibleVersions": "2+", + "fields": [ + { "name": "ElectionType", "type": "int8", "versions": "1+", + "about": "Type of elections to conduct for the partition. A value of '0' elects the preferred replica. A value of '1' elects the first live replica if there are no in-sync replica." }, + { "name": "TopicPartitions", "type": "[]TopicPartitions", "versions": "0+", "nullableVersions": "0+", + "about": "The topic partitions to elect leaders.", + "fields": [ + { "name": "Topic", "type": "string", "versions": "0+", "entityType": "topicName", "mapKey": true, + "about": "The name of a topic." }, + { "name": "Partitions", "type": "[]int32", "versions": "0+", + "about": "The partitions of this topic whose leader should be elected." } + ] + }, + { "name": "TimeoutMs", "type": "int32", "versions": "0+", "default": "60000", + "about": "The time in ms to wait for the election to complete." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/ElectLeadersResponse.json b/kafka/protocol/new/schemas/resources/ElectLeadersResponse.json new file mode 100644 index 000000000..2da4982da --- /dev/null +++ b/kafka/protocol/new/schemas/resources/ElectLeadersResponse.json @@ -0,0 +1,45 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 43, + "type": "response", + "name": "ElectLeadersResponse", + // Version 1 adds a top-level error code. + // + // Version 2 is the first flexible version. + "validVersions": "0-2", + "flexibleVersions": "2+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "1+", "ignorable": false, + "about": "The top level response error code." }, + { "name": "ReplicaElectionResults", "type": "[]ReplicaElectionResult", "versions": "0+", + "about": "The election results, or an empty array if the requester did not have permission and the request asks for all partitions.", "fields": [ + { "name": "Topic", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "PartitionResult", "type": "[]PartitionResult", "versions": "0+", + "about": "The results for each partition.", "fields": [ + { "name": "PartitionId", "type": "int32", "versions": "0+", + "about": "The partition id." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The result error, or zero if there was no error."}, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The result message, or null if there was no error."} + ]} + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/EndTxnRequest.json b/kafka/protocol/new/schemas/resources/EndTxnRequest.json new file mode 100644 index 000000000..f11c7a326 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/EndTxnRequest.json @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 26, + "type": "request", + "listeners": ["broker"], + "name": "EndTxnRequest", + // Version 1 is the same as version 0. + // + // Version 2 adds the support for new error code PRODUCER_FENCED. + // + // Version 3 enables flexible versions. + // + // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + // + // Version 5 enables bumping epoch on every transaction (KIP-890 Part 2) + "latestVersionUnstable": false, + "validVersions": "0-5", + "flexibleVersions": "3+", + "fields": [ + { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId", + "about": "The ID of the transaction to end." }, + { "name": "ProducerId", "type": "int64", "versions": "0+", "entityType": "producerId", + "about": "The producer ID." }, + { "name": "ProducerEpoch", "type": "int16", "versions": "0+", + "about": "The current epoch associated with the producer." }, + { "name": "Committed", "type": "bool", "versions": "0+", + "about": "True if the transaction was committed, false if it was aborted." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/EndTxnResponse.json b/kafka/protocol/new/schemas/resources/EndTxnResponse.json new file mode 100644 index 000000000..7f9017eda --- /dev/null +++ b/kafka/protocol/new/schemas/resources/EndTxnResponse.json @@ -0,0 +1,41 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 26, + "type": "response", + "name": "EndTxnResponse", + // Starting in version 1, on quota violation, brokers send out responses before throttling. + // + // Version 2 adds the support for new error code PRODUCER_FENCED. + // + // Version 3 enables flexible versions. + // + // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + // + // Version 5 enables bumping epoch on every transaction (KIP-890 Part 2), so producer ID and epoch are included in the response. + "validVersions": "0-5", + "flexibleVersions": "3+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." }, + { "name": "ProducerId", "type": "int64", "versions": "5+", "entityType": "producerId", "default": "-1", "ignorable": true, + "about": "The producer ID." }, + { "name": "ProducerEpoch", "type": "int16", "versions": "5+", "default": "-1", "ignorable": true, + "about": "The current epoch associated with the producer." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/FetchRequest.json b/kafka/protocol/new/schemas/resources/FetchRequest.json new file mode 100644 index 000000000..b7ad185f6 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/FetchRequest.json @@ -0,0 +1,120 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 1, + "type": "request", + "listeners": ["broker", "controller"], + "name": "FetchRequest", + // Versions 0-3 were removed in Apache Kafka 4.0, Version 4 is the new baseline. + // + // Version 1 is the same as version 0. + // Starting in Version 2, the requester must be able to handle Kafka Log + // Message format version 1. + // Version 3 adds MaxBytes. Starting in version 3, the partition ordering in + // the request is now relevant. Partitions will be processed in the order + // they appear in the request. + // + // Version 4 adds IsolationLevel. Starting in version 4, the reqestor must be + // able to handle Kafka log message format version 2. + // + // Version 5 adds LogStartOffset to indicate the earliest available offset of + // partition data that can be consumed. + // + // Version 6 is the same as version 5. + // + // Version 7 adds incremental fetch request support. + // + // Version 8 is the same as version 7. + // + // Version 9 adds CurrentLeaderEpoch, as described in KIP-320. + // + // Version 10 indicates that we can use the ZStd compression algorithm, as + // described in KIP-110. + // Version 12 adds flexible versions support as well as epoch validation through + // the `LastFetchedEpoch` field + // + // Version 13 replaces topic names with topic IDs (KIP-516). May return UNKNOWN_TOPIC_ID error code. + // + // Version 14 is the same as version 13 but it also receives a new error called OffsetMovedToTieredStorageException(KIP-405) + // + // Version 15 adds the ReplicaState which includes new field ReplicaEpoch and the ReplicaId. Also, + // deprecate the old ReplicaId field and set its default value to -1. (KIP-903) + // + // Version 16 is the same as version 15 (KIP-951). + // + // Version 17 adds directory id support from KIP-853 + "validVersions": "4-17", + "flexibleVersions": "12+", + "fields": [ + { "name": "ClusterId", "type": "string", "versions": "12+", "nullableVersions": "12+", "default": "null", + "taggedVersions": "12+", "tag": 0, "ignorable": true, + "about": "The clusterId if known. This is used to validate metadata fetches prior to broker registration." }, + { "name": "ReplicaId", "type": "int32", "versions": "0-14", "default": "-1", "entityType": "brokerId", + "about": "The broker ID of the follower, of -1 if this request is from a consumer." }, + { "name": "ReplicaState", "type": "ReplicaState", "versions": "15+", "taggedVersions": "15+", "tag": 1, + "about": "The state of the replica in the follower.", "fields": [ + { "name": "ReplicaId", "type": "int32", "versions": "15+", "default": "-1", "entityType": "brokerId", + "about": "The replica ID of the follower, or -1 if this request is from a consumer." }, + { "name": "ReplicaEpoch", "type": "int64", "versions": "15+", "default": "-1", + "about": "The epoch of this follower, or -1 if not available." } + ]}, + { "name": "MaxWaitMs", "type": "int32", "versions": "0+", + "about": "The maximum time in milliseconds to wait for the response." }, + { "name": "MinBytes", "type": "int32", "versions": "0+", + "about": "The minimum bytes to accumulate in the response." }, + { "name": "MaxBytes", "type": "int32", "versions": "3+", "default": "0x7fffffff", "ignorable": true, + "about": "The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored." }, + { "name": "IsolationLevel", "type": "int8", "versions": "4+", "default": "0", "ignorable": true, + "about": "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records." }, + { "name": "SessionId", "type": "int32", "versions": "7+", "default": "0", "ignorable": true, + "about": "The fetch session ID." }, + { "name": "SessionEpoch", "type": "int32", "versions": "7+", "default": "-1", "ignorable": true, + "about": "The fetch session epoch, which is used for ordering requests in a session." }, + { "name": "Topics", "type": "[]FetchTopic", "versions": "0+", + "about": "The topics to fetch.", "fields": [ + { "name": "Topic", "type": "string", "versions": "0-12", "entityType": "topicName", "ignorable": true, + "about": "The name of the topic to fetch." }, + { "name": "TopicId", "type": "uuid", "versions": "13+", "ignorable": true, "about": "The unique topic ID."}, + { "name": "Partitions", "type": "[]FetchPartition", "versions": "0+", + "about": "The partitions to fetch.", "fields": [ + { "name": "Partition", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "CurrentLeaderEpoch", "type": "int32", "versions": "9+", "default": "-1", "ignorable": true, + "about": "The current leader epoch of the partition." }, + { "name": "FetchOffset", "type": "int64", "versions": "0+", + "about": "The message offset." }, + { "name": "LastFetchedEpoch", "type": "int32", "versions": "12+", "default": "-1", "ignorable": false, + "about": "The epoch of the last fetched record or -1 if there is none."}, + { "name": "LogStartOffset", "type": "int64", "versions": "5+", "default": "-1", "ignorable": true, + "about": "The earliest available offset of the follower replica. The field is only used when the request is sent by the follower."}, + { "name": "PartitionMaxBytes", "type": "int32", "versions": "0+", + "about": "The maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored." }, + { "name": "ReplicaDirectoryId", "type": "uuid", "versions": "17+", "taggedVersions": "17+", "tag": 0, "ignorable": true, + "about": "The directory id of the follower fetching." } + ]} + ]}, + { "name": "ForgottenTopicsData", "type": "[]ForgottenTopic", "versions": "7+", "ignorable": false, + "about": "In an incremental fetch request, the partitions to remove.", "fields": [ + { "name": "Topic", "type": "string", "versions": "7-12", "entityType": "topicName", "ignorable": true, + "about": "The topic name." }, + { "name": "TopicId", "type": "uuid", "versions": "13+", "ignorable": true, "about": "The unique topic ID."}, + { "name": "Partitions", "type": "[]int32", "versions": "7+", + "about": "The partitions indexes to forget." } + ]}, + { "name": "RackId", "type": "string", "versions": "11+", "default": "", "ignorable": true, + "about": "Rack ID of the consumer making this request."} + ] +} diff --git a/kafka/protocol/new/schemas/resources/FetchResponse.json b/kafka/protocol/new/schemas/resources/FetchResponse.json new file mode 100644 index 000000000..dc8d35175 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/FetchResponse.json @@ -0,0 +1,122 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 1, + "type": "response", + "name": "FetchResponse", + // Versions 0-3 were removed in Apache Kafka 4.0, Version 4 is the new baseline. + // + // Version 1 adds throttle time. Version 2 and 3 are the same as version 1. + // + // Version 4 adds features for transactional consumption. + // + // Version 5 adds LogStartOffset to indicate the earliest available offset of + // partition data that can be consumed. + // + // Starting in version 6, we may return KAFKA_STORAGE_ERROR as an error code. + // + // Version 7 adds incremental fetch request support. + // + // Starting in version 8, on quota violation, brokers send out responses before throttling. + // + // Version 9 is the same as version 8. + // + // Version 10 indicates that the response data can use the ZStd compression + // algorithm, as described in KIP-110. + // Version 12 adds support for flexible versions, epoch detection through the `TruncationOffset` field, + // and leader discovery through the `CurrentLeader` field + // + // Version 13 replaces the topic name field with topic ID (KIP-516). + // + // Version 14 is the same as version 13 but it also receives a new error called OffsetMovedToTieredStorageException (KIP-405) + // + // Version 15 is the same as version 14 (KIP-903). + // + // Version 16 adds the 'NodeEndpoints' field (KIP-951). + // + // Version 17 no changes to the response (KIP-853). + "validVersions": "4-17", + "flexibleVersions": "12+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "7+", "ignorable": true, + "about": "The top level response error code." }, + { "name": "SessionId", "type": "int32", "versions": "7+", "default": "0", "ignorable": false, + "about": "The fetch session ID, or 0 if this is not part of a fetch session." }, + { "name": "Responses", "type": "[]FetchableTopicResponse", "versions": "0+", + "about": "The response topics.", "fields": [ + { "name": "Topic", "type": "string", "versions": "0-12", "ignorable": true, "entityType": "topicName", + "about": "The topic name." }, + { "name": "TopicId", "type": "uuid", "versions": "13+", "ignorable": true, "about": "The unique topic ID."}, + { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", + "about": "The topic partitions.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no fetch error." }, + { "name": "HighWatermark", "type": "int64", "versions": "0+", + "about": "The current high water mark." }, + { "name": "LastStableOffset", "type": "int64", "versions": "4+", "default": "-1", "ignorable": true, + "about": "The last stable offset (or LSO) of the partition. This is the last offset such that the state of all transactional records prior to this offset have been decided (ABORTED or COMMITTED)." }, + { "name": "LogStartOffset", "type": "int64", "versions": "5+", "default": "-1", "ignorable": true, + "about": "The current log start offset." }, + { "name": "DivergingEpoch", "type": "EpochEndOffset", "versions": "12+", "taggedVersions": "12+", "tag": 0, + "about": "In case divergence is detected based on the `LastFetchedEpoch` and `FetchOffset` in the request, this field indicates the largest epoch and its end offset such that subsequent records are known to diverge.", "fields": [ + { "name": "Epoch", "type": "int32", "versions": "12+", "default": "-1", + "about": "The largest epoch." }, + { "name": "EndOffset", "type": "int64", "versions": "12+", "default": "-1", + "about": "The end offset of the epoch." } + ]}, + { "name": "CurrentLeader", "type": "LeaderIdAndEpoch", + "versions": "12+", "taggedVersions": "12+", "tag": 1, + "about": "The current leader of the partition.", "fields": [ + { "name": "LeaderId", "type": "int32", "versions": "12+", "default": "-1", "entityType": "brokerId", + "about": "The ID of the current leader or -1 if the leader is unknown."}, + { "name": "LeaderEpoch", "type": "int32", "versions": "12+", "default": "-1", + "about": "The latest known leader epoch." } + ]}, + { "name": "SnapshotId", "type": "SnapshotId", + "versions": "12+", "taggedVersions": "12+", "tag": 2, + "about": "In the case of fetching an offset less than the LogStartOffset, this is the end offset and epoch that should be used in the FetchSnapshot request.", "fields": [ + { "name": "EndOffset", "type": "int64", "versions": "0+", "default": "-1", + "about": "The end offset of the epoch." }, + { "name": "Epoch", "type": "int32", "versions": "0+", "default": "-1", + "about": "The largest epoch." } + ]}, + { "name": "AbortedTransactions", "type": "[]AbortedTransaction", "versions": "4+", "nullableVersions": "4+", "ignorable": true, + "about": "The aborted transactions.", "fields": [ + { "name": "ProducerId", "type": "int64", "versions": "4+", "entityType": "producerId", + "about": "The producer id associated with the aborted transaction." }, + { "name": "FirstOffset", "type": "int64", "versions": "4+", + "about": "The first offset in the aborted transaction." } + ]}, + { "name": "PreferredReadReplica", "type": "int32", "versions": "11+", "default": "-1", "ignorable": false, "entityType": "brokerId", + "about": "The preferred read replica for the consumer to use on its next fetch request."}, + { "name": "Records", "type": "records", "versions": "0+", "nullableVersions": "0+", "about": "The record data."} + ]} + ]}, + { "name": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "16+", "taggedVersions": "16+", "tag": 0, + "about": "Endpoints for all current-leaders enumerated in PartitionData, with errors NOT_LEADER_OR_FOLLOWER & FENCED_LEADER_EPOCH.", "fields": [ + { "name": "NodeId", "type": "int32", "versions": "16+", + "mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node."}, + { "name": "Host", "type": "string", "versions": "16+", "about": "The node's hostname." }, + { "name": "Port", "type": "int32", "versions": "16+", "about": "The node's port." }, + { "name": "Rack", "type": "string", "versions": "16+", "nullableVersions": "16+", "default": "null", + "about": "The rack of the node, or null if it has not been assigned to a rack." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/FindCoordinatorRequest.json b/kafka/protocol/new/schemas/resources/FindCoordinatorRequest.json new file mode 100644 index 000000000..2807f40c8 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/FindCoordinatorRequest.json @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 10, + "type": "request", + "listeners": ["broker"], + "name": "FindCoordinatorRequest", + // Version 1 adds KeyType. + // + // Version 2 is the same as version 1. + // + // Version 3 is the first flexible version. + // + // Version 4 adds support for batching via CoordinatorKeys (KIP-699) + // + // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + // + // Version 6 adds support for share groups (KIP-932). + // For key type SHARE (2), the coordinator key format is "groupId:topicId:partition". + "validVersions": "0-6", + "flexibleVersions": "3+", + "fields": [ + { "name": "Key", "type": "string", "versions": "0-3", + "about": "The coordinator key." }, + { "name": "KeyType", "type": "int8", "versions": "1+", "default": "0", "ignorable": false, + "about": "The coordinator key type. (group, transaction, share)." }, + { "name": "CoordinatorKeys", "type": "[]string", "versions": "4+", + "about": "The coordinator keys." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/FindCoordinatorResponse.json b/kafka/protocol/new/schemas/resources/FindCoordinatorResponse.json new file mode 100644 index 000000000..40c43b65f --- /dev/null +++ b/kafka/protocol/new/schemas/resources/FindCoordinatorResponse.json @@ -0,0 +1,58 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 10, + "type": "response", + "name": "FindCoordinatorResponse", + // Version 1 adds throttle time and error messages. + // + // Starting in version 2, on quota violation, brokers send out responses before throttling. + // + // Version 3 is the first flexible version. + // + // Version 4 adds support for batching via Coordinators (KIP-699) + // + // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + // + // Version 6 adds support for share groups (KIP-932). + "validVersions": "0-6", + "flexibleVersions": "3+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0-3", + "about": "The error code, or 0 if there was no error." }, + { "name": "ErrorMessage", "type": "string", "versions": "1-3", "nullableVersions": "1-3", "ignorable": true, + "about": "The error message, or null if there was no error." }, + { "name": "NodeId", "type": "int32", "versions": "0-3", "entityType": "brokerId", + "about": "The node id." }, + { "name": "Host", "type": "string", "versions": "0-3", + "about": "The host name." }, + { "name": "Port", "type": "int32", "versions": "0-3", + "about": "The port." }, + { "name": "Coordinators", "type": "[]Coordinator", "versions": "4+", "about": "Each coordinator result in the response.", "fields": [ + { "name": "Key", "type": "string", "versions": "4+", "about": "The coordinator key." }, + { "name": "NodeId", "type": "int32", "versions": "4+", "entityType": "brokerId", + "about": "The node id." }, + { "name": "Host", "type": "string", "versions": "4+", "about": "The host name." }, + { "name": "Port", "type": "int32", "versions": "4+", "about": "The port." }, + { "name": "ErrorCode", "type": "int16", "versions": "4+", + "about": "The error code, or 0 if there was no error." }, + { "name": "ErrorMessage", "type": "string", "versions": "4+", "nullableVersions": "4+", "ignorable": true, + "about": "The error message, or null if there was no error." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/HeartbeatRequest.json b/kafka/protocol/new/schemas/resources/HeartbeatRequest.json new file mode 100644 index 000000000..57ef18e92 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/HeartbeatRequest.json @@ -0,0 +1,39 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 12, + "type": "request", + "listeners": ["broker"], + "name": "HeartbeatRequest", + // Version 1 and version 2 are the same as version 0. + // + // Starting from version 3, we add a new field called groupInstanceId to indicate member identity across restarts. + // + // Version 4 is the first flexible version. + "validVersions": "0-4", + "flexibleVersions": "4+", + "fields": [ + { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", + "about": "The group id." }, + { "name": "GenerationId", "type": "int32", "versions": "0+", + "about": "The generation of the group." }, + { "name": "MemberId", "type": "string", "versions": "0+", + "about": "The member ID." }, + { "name": "GroupInstanceId", "type": "string", "versions": "3+", + "nullableVersions": "3+", "default": "null", + "about": "The unique identifier of the consumer instance provided by end user." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/HeartbeatResponse.json b/kafka/protocol/new/schemas/resources/HeartbeatResponse.json new file mode 100644 index 000000000..280ba1103 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/HeartbeatResponse.json @@ -0,0 +1,35 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 12, + "type": "response", + "name": "HeartbeatResponse", + // Version 1 adds throttle time. + // + // Starting in version 2, on quota violation, brokers send out responses before throttling. + // + // Starting from version 3, heartbeatRequest supports a new field called groupInstanceId to indicate member identity across restarts. + // + // Version 4 is the first flexible version. + "validVersions": "0-4", + "flexibleVersions": "4+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/InitProducerIdRequest.json b/kafka/protocol/new/schemas/resources/InitProducerIdRequest.json new file mode 100644 index 000000000..f1700f3a3 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/InitProducerIdRequest.json @@ -0,0 +1,42 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 22, + "type": "request", + "listeners": ["broker"], + "name": "InitProducerIdRequest", + // Version 1 is the same as version 0. + // + // Version 2 is the first flexible version. + // + // Version 3 adds ProducerId and ProducerEpoch, allowing producers to try to resume after an INVALID_PRODUCER_EPOCH error + // + // Version 4 adds the support for new error code PRODUCER_FENCED. + // + // Verison 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-5", + "flexibleVersions": "2+", + "fields": [ + { "name": "TransactionalId", "type": "string", "versions": "0+", "nullableVersions": "0+", "entityType": "transactionalId", + "about": "The transactional id, or null if the producer is not transactional." }, + { "name": "TransactionTimeoutMs", "type": "int32", "versions": "0+", + "about": "The time in ms to wait before aborting idle transactions sent by this producer. This is only relevant if a TransactionalId has been defined." }, + { "name": "ProducerId", "type": "int64", "versions": "3+", "default": "-1", "entityType": "producerId", + "about": "The producer id. This is used to disambiguate requests if a transactional id is reused following its expiration." }, + { "name": "ProducerEpoch", "type": "int16", "versions": "3+", "default": "-1", + "about": "The producer's current epoch. This will be checked against the producer epoch on the broker, and the request will return an error if they do not match." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/InitProducerIdResponse.json b/kafka/protocol/new/schemas/resources/InitProducerIdResponse.json new file mode 100644 index 000000000..c5dfec6e3 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/InitProducerIdResponse.json @@ -0,0 +1,41 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 22, + "type": "response", + "name": "InitProducerIdResponse", + // Starting in version 1, on quota violation, brokers send out responses before throttling. + // + // Version 2 is the first flexible version. + // + // Version 3 is the same as version 2. + // + // Version 4 adds the support for new error code PRODUCER_FENCED. + // + // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + "validVersions": "0-5", + "flexibleVersions": "2+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." }, + { "name": "ProducerId", "type": "int64", "versions": "0+", "entityType": "producerId", + "default": -1, "about": "The current producer id." }, + { "name": "ProducerEpoch", "type": "int16", "versions": "0+", + "about": "The current epoch associated with the producer id." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/JoinGroupRequest.json b/kafka/protocol/new/schemas/resources/JoinGroupRequest.json new file mode 100644 index 000000000..41d7c1acb --- /dev/null +++ b/kafka/protocol/new/schemas/resources/JoinGroupRequest.json @@ -0,0 +1,65 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 11, + "type": "request", + "listeners": ["broker"], + "name": "JoinGroupRequest", + // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline. + // + // Version 1 adds RebalanceTimeoutMs. Version 2 and 3 are the same as version 1. + // + // Starting from version 4, the client needs to issue a second request to join group + // + // Starting from version 5, we add a new field called groupInstanceId to indicate member identity across restarts. + // with assigned id. + // + // Version 6 is the first flexible version. + // + // Version 7 is the same as version 6. + // + // Version 8 adds the Reason field (KIP-800). + // + // Version 9 is the same as version 8. + "validVersions": "2-9", + "flexibleVersions": "6+", + "fields": [ + { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", + "about": "The group identifier." }, + { "name": "SessionTimeoutMs", "type": "int32", "versions": "0+", + "about": "The coordinator considers the consumer dead if it receives no heartbeat after this timeout in milliseconds." }, + // Note: if RebalanceTimeoutMs is not present, SessionTimeoutMs should be + // used instead. The default of -1 here is just intended as a placeholder. + { "name": "RebalanceTimeoutMs", "type": "int32", "versions": "1+", "default": "-1", "ignorable": true, + "about": "The maximum time in milliseconds that the coordinator will wait for each member to rejoin when rebalancing the group." }, + { "name": "MemberId", "type": "string", "versions": "0+", + "about": "The member id assigned by the group coordinator." }, + { "name": "GroupInstanceId", "type": "string", "versions": "5+", + "nullableVersions": "5+", "default": "null", + "about": "The unique identifier of the consumer instance provided by end user." }, + { "name": "ProtocolType", "type": "string", "versions": "0+", + "about": "The unique name the for class of protocols implemented by the group we want to join." }, + { "name": "Protocols", "type": "[]JoinGroupRequestProtocol", "versions": "0+", + "about": "The list of protocols that the member supports.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, + "about": "The protocol name." }, + { "name": "Metadata", "type": "bytes", "versions": "0+", + "about": "The protocol metadata." } + ]}, + { "name": "Reason", "type": "string", "versions": "8+", "nullableVersions": "8+", "default": "null", "ignorable": true, + "about": "The reason why the member (re-)joins the group." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/JoinGroupResponse.json b/kafka/protocol/new/schemas/resources/JoinGroupResponse.json new file mode 100644 index 000000000..364309596 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/JoinGroupResponse.json @@ -0,0 +1,71 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 11, + "type": "response", + "name": "JoinGroupResponse", + // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline. + // + // Version 1 is the same as version 0. + // + // Version 2 adds throttle time. + // + // Starting in version 3, on quota violation, brokers send out responses before throttling. + // + // Starting in version 4, the client needs to issue a second request to join group + // with assigned id. + // + // Version 5 is bumped to apply group.instance.id to identify member across restarts. + // + // Version 6 is the first flexible version. + // + // Starting from version 7, the broker sends back the Protocol Type to the client (KIP-559). + // + // Version 8 is the same as version 7. + // + // Version 9 adds the SkipAssignment field. + "validVersions": "2-9", + "flexibleVersions": "6+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." }, + { "name": "GenerationId", "type": "int32", "versions": "0+", "default": "-1", + "about": "The generation ID of the group." }, + { "name": "ProtocolType", "type": "string", "versions": "7+", + "nullableVersions": "7+", "default": "null", "ignorable": true, + "about": "The group protocol name." }, + { "name": "ProtocolName", "type": "string", "versions": "0+", "nullableVersions": "7+", + "about": "The group protocol selected by the coordinator." }, + { "name": "Leader", "type": "string", "versions": "0+", + "about": "The leader of the group." }, + { "name": "SkipAssignment", "type": "bool", "versions": "9+", "default": "false", + "about": "True if the leader must skip running the assignment." }, + { "name": "MemberId", "type": "string", "versions": "0+", + "about": "The member ID assigned by the group coordinator." }, + { "name": "Members", "type": "[]JoinGroupResponseMember", "versions": "0+", + "about": "The group members.", "fields": [ + { "name": "MemberId", "type": "string", "versions": "0+", + "about": "The group member ID." }, + { "name": "GroupInstanceId", "type": "string", "versions": "5+", "ignorable": true, + "nullableVersions": "5+", "default": "null", + "about": "The unique identifier of the consumer instance provided by end user." }, + { "name": "Metadata", "type": "bytes", "versions": "0+", + "about": "The group member metadata." }] + } + ] +} diff --git a/kafka/protocol/new/schemas/resources/LeaveGroupRequest.json b/kafka/protocol/new/schemas/resources/LeaveGroupRequest.json new file mode 100644 index 000000000..929f4fb46 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/LeaveGroupRequest.json @@ -0,0 +1,47 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 13, + "type": "request", + "listeners": ["broker"], + "name": "LeaveGroupRequest", + // Version 1 and 2 are the same as version 0. + // + // Version 3 defines batch processing scheme with group.instance.id + member.id for identity + // + // Version 4 is the first flexible version. + // + // Version 5 adds the Reason field (KIP-800). + "validVersions": "0-5", + "flexibleVersions": "4+", + "fields": [ + { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", + "about": "The ID of the group to leave." }, + { "name": "MemberId", "type": "string", "versions": "0-2", + "about": "The member ID to remove from the group." }, + { "name": "Members", "type": "[]MemberIdentity", "versions": "3+", + "about": "List of leaving member identities.", "fields": [ + { "name": "MemberId", "type": "string", "versions": "3+", + "about": "The member ID to remove from the group." }, + { "name": "GroupInstanceId", "type": "string", + "versions": "3+", "nullableVersions": "3+", "default": "null", + "about": "The group instance ID to remove from the group." }, + { "name": "Reason", "type": "string", + "versions": "5+", "nullableVersions": "5+", "default": "null", "ignorable": true, + "about": "The reason why the member left the group." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/LeaveGroupResponse.json b/kafka/protocol/new/schemas/resources/LeaveGroupResponse.json new file mode 100644 index 000000000..d3c8784b0 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/LeaveGroupResponse.json @@ -0,0 +1,47 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 13, + "type": "response", + "name": "LeaveGroupResponse", + // Version 1 adds the throttle time. + // + // Starting in version 2, on quota violation, brokers send out responses before throttling. + // + // Starting in version 3, we will make leave group request into batch mode and add group.instance.id. + // + // Version 4 is the first flexible version. + // + // Version 5 is the same as version 4. + "validVersions": "0-5", + "flexibleVersions": "4+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." }, + + { "name": "Members", "type": "[]MemberResponse", "versions": "3+", + "about": "List of leaving member responses.", "fields": [ + { "name": "MemberId", "type": "string", "versions": "3+", + "about": "The member ID to remove from the group." }, + { "name": "GroupInstanceId", "type": "string", "versions": "3+", "nullableVersions": "3+", + "about": "The group instance ID to remove from the group." }, + { "name": "ErrorCode", "type": "int16", "versions": "3+", + "about": "The error code, or 0 if there was no error." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/ListGroupsRequest.json b/kafka/protocol/new/schemas/resources/ListGroupsRequest.json new file mode 100644 index 000000000..cbc791e0a --- /dev/null +++ b/kafka/protocol/new/schemas/resources/ListGroupsRequest.json @@ -0,0 +1,36 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 16, + "type": "request", + "listeners": ["broker"], + "name": "ListGroupsRequest", + // Version 1 and 2 are the same as version 0. + // + // Version 3 is the first flexible version. + // + // Version 4 adds the StatesFilter field (KIP-518). + // + // Version 5 adds the TypesFilter field (KIP-848). + "validVersions": "0-5", + "flexibleVersions": "3+", + "fields": [ + { "name": "StatesFilter", "type": "[]string", "versions": "4+", + "about": "The states of the groups we want to list. If empty, all groups are returned with their state." }, + { "name": "TypesFilter", "type": "[]string", "versions": "5+", + "about": "The types of the groups we want to list. If empty, all groups are returned with their type." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/ListGroupsResponse.json b/kafka/protocol/new/schemas/resources/ListGroupsResponse.json new file mode 100644 index 000000000..fc4077c08 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/ListGroupsResponse.json @@ -0,0 +1,49 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 16, + "type": "response", + "name": "ListGroupsResponse", + // Version 1 adds the throttle time. + // + // Starting in version 2, on quota violation, brokers send out + // responses before throttling. + // + // Version 3 is the first flexible version. + // + // Version 4 adds the GroupState field (KIP-518). + // + // Version 5 adds the GroupType field (KIP-848). + "validVersions": "0-5", + "flexibleVersions": "3+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." }, + { "name": "Groups", "type": "[]ListedGroup", "versions": "0+", + "about": "Each group in the response.", "fields": [ + { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", + "about": "The group ID." }, + { "name": "ProtocolType", "type": "string", "versions": "0+", + "about": "The group protocol type." }, + { "name": "GroupState", "type": "string", "versions": "4+", "ignorable": true, + "about": "The group state name." }, + { "name": "GroupType", "type": "string", "versions": "5+", "ignorable": true, + "about": "The group type name." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/ListOffsetsRequest.json b/kafka/protocol/new/schemas/resources/ListOffsetsRequest.json new file mode 100644 index 000000000..6f8ff7d6c --- /dev/null +++ b/kafka/protocol/new/schemas/resources/ListOffsetsRequest.json @@ -0,0 +1,68 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 2, + "type": "request", + "listeners": ["broker"], + "name": "ListOffsetsRequest", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // + // Version 1 removes MaxNumOffsets. From this version forward, only a single + // offset can be returned. + // + // Version 2 adds the isolation level, which is used for transactional reads. + // + // Version 3 is the same as version 2. + // + // Version 4 adds the current leader epoch, which is used for fencing. + // + // Version 5 is the same as version 4. + // + // Version 6 enables flexible versions. + // + // Version 7 enables listing offsets by max timestamp (KIP-734). + // + // Version 8 enables listing offsets by local log start offset (KIP-405). + // + // Version 9 enables listing offsets by last tiered offset (KIP-1005). + // + // Version 10 enables async remote list offsets support (KIP-1075) + "validVersions": "1-10", + "flexibleVersions": "6+", + "latestVersionUnstable": false, + "fields": [ + { "name": "ReplicaId", "type": "int32", "versions": "0+", "entityType": "brokerId", + "about": "The broker ID of the requester, or -1 if this request is being made by a normal consumer." }, + { "name": "IsolationLevel", "type": "int8", "versions": "2+", + "about": "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED (isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), non-transactional and COMMITTED transactional records are visible. To be more concrete, READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), and enables the inclusion of the list of aborted transactions in the result, which allows consumers to discard ABORTED transactional records." }, + { "name": "Topics", "type": "[]ListOffsetsTopic", "versions": "0+", + "about": "Each topic in the request.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]ListOffsetsPartition", "versions": "0+", + "about": "Each partition in the request.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "CurrentLeaderEpoch", "type": "int32", "versions": "4+", "default": "-1", "ignorable": true, + "about": "The current leader epoch." }, + { "name": "Timestamp", "type": "int64", "versions": "0+", + "about": "The current timestamp." } + ]} + ]}, + { "name": "TimeoutMs", "type": "int32", "versions": "10+", "ignorable": true, + "about": "The timeout to await a response in milliseconds for requests that require reading from remote storage for topics enabled with tiered storage." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/ListOffsetsResponse.json b/kafka/protocol/new/schemas/resources/ListOffsetsResponse.json new file mode 100644 index 000000000..7f9588847 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/ListOffsetsResponse.json @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 2, + "type": "response", + "name": "ListOffsetsResponse", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // + // Version 1 removes the offsets array in favor of returning a single offset. + // Version 1 also adds the timestamp associated with the returned offset. + // + // Version 2 adds the throttle time. + // + // Starting in version 3, on quota violation, brokers send out responses before throttling. + // + // Version 4 adds the leader epoch, which is used for fencing. + // + // Version 5 adds a new error code, OFFSET_NOT_AVAILABLE. + // + // Version 6 enables flexible versions. + // + // Version 7 is the same as version 6 (KIP-734). + // + // Version 8 enables listing offsets by local log start offset. + // This is the earliest log start offset in the local log. (KIP-405). + // + // Version 9 enables listing offsets by last tiered offset (KIP-1005). + // + // Version 10 enables async remote list offsets support (KIP-1075) + "validVersions": "1-10", + "flexibleVersions": "6+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Topics", "type": "[]ListOffsetsTopicResponse", "versions": "0+", + "about": "Each topic in the response.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]ListOffsetsPartitionResponse", "versions": "0+", + "about": "Each partition in the response.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The partition error code, or 0 if there was no error." }, + { "name": "Timestamp", "type": "int64", "versions": "1+", "default": "-1", "ignorable": false, + "about": "The timestamp associated with the returned offset." }, + { "name": "Offset", "type": "int64", "versions": "1+", "default": "-1", "ignorable": false, + "about": "The returned offset." }, + { "name": "LeaderEpoch", "type": "int32", "versions": "4+", "default": "-1", + "about": "The leader epoch associated with the returned offset."} + ]} + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/MetadataRequest.json b/kafka/protocol/new/schemas/resources/MetadataRequest.json new file mode 100644 index 000000000..349f88b7c --- /dev/null +++ b/kafka/protocol/new/schemas/resources/MetadataRequest.json @@ -0,0 +1,56 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 3, + "type": "request", + "listeners": ["broker"], + "name": "MetadataRequest", + "validVersions": "0-13", + "flexibleVersions": "9+", + "fields": [ + // In version 0, an empty array indicates "request metadata for all topics." In version 1 and + // higher, an empty array indicates "request metadata for no topics," and a null array is used to + // indicate "request metadata for all topics." + // + // Version 2 and 3 are the same as version 1. + // + // Version 4 adds AllowAutoTopicCreation. + // + // Starting in version 8, authorized operations can be requested for cluster and topic resource. + // + // Version 9 is the first flexible version. + // + // Version 10 adds topicId and allows name field to be null. However, this functionality was not implemented on the server. + // Versions 10 and 11 should not use the topicId field or set topic name to null. + // + // Version 11 deprecates IncludeClusterAuthorizedOperations field. This is now exposed + // by the DescribeCluster API (KIP-700). + // Version 12 supports topic Id. + // Version 13 supports top-level error code in the response. + { "name": "Topics", "type": "[]MetadataRequestTopic", "versions": "0+", "nullableVersions": "1+", + "about": "The topics to fetch metadata for.", "fields": [ + { "name": "TopicId", "type": "uuid", "versions": "10+", "ignorable": true, "about": "The topic id." }, + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", "nullableVersions": "10+", + "about": "The topic name." } + ]}, + { "name": "AllowAutoTopicCreation", "type": "bool", "versions": "4+", "default": "true", "ignorable": false, + "about": "If this is true, the broker may auto-create topics that we requested which do not already exist, if it is configured to do so." }, + { "name": "IncludeClusterAuthorizedOperations", "type": "bool", "versions": "8-10", + "about": "Whether to include cluster authorized operations." }, + { "name": "IncludeTopicAuthorizedOperations", "type": "bool", "versions": "8+", + "about": "Whether to include topic authorized operations." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/MetadataResponse.json b/kafka/protocol/new/schemas/resources/MetadataResponse.json new file mode 100644 index 000000000..07ee7010e --- /dev/null +++ b/kafka/protocol/new/schemas/resources/MetadataResponse.json @@ -0,0 +1,101 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 3, + "type": "response", + "name": "MetadataResponse", + // Version 1 adds fields for the rack of each broker, the controller id, and whether or not the topic is internal. + // + // Version 2 adds the cluster ID field. + // + // Version 3 adds the throttle time. + // + // Version 4 is the same as version 3. + // + // Version 5 adds a per-partition offline_replicas field. This field specifies + // the list of replicas that are offline. + // + // Starting in version 6, on quota violation, brokers send out responses before throttling. + // + // Version 7 adds the leader epoch to the partition metadata. + // + // Starting in version 8, brokers can send authorized operations for topic and cluster. + // + // Version 9 is the first flexible version. + // + // Version 10 adds topicId. + // + // Version 11 deprecates ClusterAuthorizedOperations. This is now exposed + // by the DescribeCluster API (KIP-700). + // Version 12 supports topicId. + // Version 13 supports top-level error code in the response. + "validVersions": "0-13", + "flexibleVersions": "9+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "3+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Brokers", "type": "[]MetadataResponseBroker", "versions": "0+", + "about": "A list of brokers present in the cluster.", "fields": [ + { "name": "NodeId", "type": "int32", "versions": "0+", "mapKey": true, "entityType": "brokerId", + "about": "The broker ID." }, + { "name": "Host", "type": "string", "versions": "0+", + "about": "The broker hostname." }, + { "name": "Port", "type": "int32", "versions": "0+", + "about": "The broker port." }, + { "name": "Rack", "type": "string", "versions": "1+", "nullableVersions": "1+", "ignorable": true, "default": "null", + "about": "The rack of the broker, or null if it has not been assigned to a rack." } + ]}, + { "name": "ClusterId", "type": "string", "nullableVersions": "2+", "versions": "2+", "ignorable": true, "default": "null", + "about": "The cluster ID that responding broker belongs to." }, + { "name": "ControllerId", "type": "int32", "versions": "1+", "default": "-1", "ignorable": true, "entityType": "brokerId", + "about": "The ID of the controller broker." }, + { "name": "Topics", "type": "[]MetadataResponseTopic", "versions": "0+", + "about": "Each topic in the response.", "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The topic error, or 0 if there was no error." }, + { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, "entityType": "topicName", "nullableVersions": "12+", + "about": "The topic name. Null for non-existing topics queried by ID. This is never null when ErrorCode is zero. One of Name and TopicId is always populated." }, + { "name": "TopicId", "type": "uuid", "versions": "10+", "ignorable": true, + "about": "The topic id. Zero for non-existing topics queried by name. This is never zero when ErrorCode is zero. One of Name and TopicId is always populated." }, + { "name": "IsInternal", "type": "bool", "versions": "1+", "default": "false", "ignorable": true, + "about": "True if the topic is internal." }, + { "name": "Partitions", "type": "[]MetadataResponsePartition", "versions": "0+", + "about": "Each partition in the topic.", "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The partition error, or 0 if there was no error." }, + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "LeaderId", "type": "int32", "versions": "0+", "entityType": "brokerId", + "about": "The ID of the leader broker." }, + { "name": "LeaderEpoch", "type": "int32", "versions": "7+", "default": "-1", "ignorable": true, + "about": "The leader epoch of this partition." }, + { "name": "ReplicaNodes", "type": "[]int32", "versions": "0+", "entityType": "brokerId", + "about": "The set of all nodes that host this partition." }, + { "name": "IsrNodes", "type": "[]int32", "versions": "0+", "entityType": "brokerId", + "about": "The set of nodes that are in sync with the leader for this partition." }, + { "name": "OfflineReplicas", "type": "[]int32", "versions": "5+", "ignorable": true, "entityType": "brokerId", + "about": "The set of offline replicas of this partition." } + ]}, + { "name": "TopicAuthorizedOperations", "type": "int32", "versions": "8+", "default": "-2147483648", + "about": "32-bit bitfield to represent authorized operations for this topic." } + ]}, + { "name": "ClusterAuthorizedOperations", "type": "int32", "versions": "8-10", "default": "-2147483648", + "about": "32-bit bitfield to represent authorized operations for this cluster." }, + { "name": "ErrorCode", "type": "int16", "versions": "13+", "ignorable": true, + "about": "The top-level error code, or 0 if there was no error." } + + ] +} diff --git a/kafka/protocol/new/schemas/resources/OffsetCommitRequest.json b/kafka/protocol/new/schemas/resources/OffsetCommitRequest.json new file mode 100644 index 000000000..348ed2b90 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/OffsetCommitRequest.json @@ -0,0 +1,70 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 8, + "type": "request", + "listeners": ["broker"], + "name": "OffsetCommitRequest", + // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline. + // + // Version 1 adds timestamp and group membership information, as well as the commit timestamp. + // + // Version 2 adds retention time. It removes the commit timestamp added in version 1. + // + // Version 3 and 4 are the same as version 2. + // + // Version 5 removes the retention time, which is now controlled only by a broker configuration. + // + // Version 6 adds the leader epoch for fencing. + // + // version 7 adds a new field called groupInstanceId to indicate member identity across restarts. + // + // Version 8 is the first flexible version. + // + // Version 9 is the first version that can be used with the new consumer group protocol (KIP-848). The + // request is the same as version 8. + "validVersions": "2-9", + "flexibleVersions": "8+", + "fields": [ + { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", + "about": "The unique group identifier." }, + { "name": "GenerationIdOrMemberEpoch", "type": "int32", "versions": "1+", "default": "-1", "ignorable": true, + "about": "The generation of the group if using the classic group protocol or the member epoch if using the consumer protocol." }, + { "name": "MemberId", "type": "string", "versions": "1+", "ignorable": true, + "about": "The member ID assigned by the group coordinator." }, + { "name": "GroupInstanceId", "type": "string", "versions": "7+", + "nullableVersions": "7+", "default": "null", + "about": "The unique identifier of the consumer instance provided by end user." }, + { "name": "RetentionTimeMs", "type": "int64", "versions": "2-4", "default": "-1", "ignorable": true, + "about": "The time period in ms to retain the offset." }, + { "name": "Topics", "type": "[]OffsetCommitRequestTopic", "versions": "0+", + "about": "The topics to commit offsets for.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]OffsetCommitRequestPartition", "versions": "0+", + "about": "Each partition to commit offsets for.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "CommittedOffset", "type": "int64", "versions": "0+", + "about": "The message offset to be committed." }, + { "name": "CommittedLeaderEpoch", "type": "int32", "versions": "6+", "default": "-1", "ignorable": true, + "about": "The leader epoch of this partition." }, + { "name": "CommittedMetadata", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "Any associated metadata the client wants to keep." } + ]} + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/OffsetCommitResponse.json b/kafka/protocol/new/schemas/resources/OffsetCommitResponse.json new file mode 100644 index 000000000..0cccd6481 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/OffsetCommitResponse.json @@ -0,0 +1,66 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 8, + "type": "response", + "name": "OffsetCommitResponse", + // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline. + // + // Versions 1 and 2 are the same as version 0. + // + // Version 3 adds the throttle time to the response. + // + // Starting in version 4, on quota violation, brokers send out responses before throttling. + // + // Versions 5 and 6 are the same as version 4. + // + // Version 7 offsetCommitRequest supports a new field called groupInstanceId to indicate member identity across restarts. + // + // Version 8 is the first flexible version. + // + // Version 9 is the first version that can be used with the new consumer group protocol (KIP-848). The response is + // the same as version 8 but can return STALE_MEMBER_EPOCH when the new consumer group protocol is used and + // GROUP_ID_NOT_FOUND when the group does not exist for both protocols. + "validVersions": "2-9", + "flexibleVersions": "8+", + // Supported errors: + // - GROUP_AUTHORIZATION_FAILED (version 0+) + // - NOT_COORDINATOR (version 0+) + // - COORDINATOR_NOT_AVAILABLE (version 0+) + // - COORDINATOR_LOAD_IN_PROGRESS (version 0+) + // - ILLEGAL_GENERATION (version 1+) + // - UNKNOWN_MEMBER_ID (version 1+) + // - INVALID_COMMIT_OFFSET_SIZE (version 0+) + // - FENCED_MEMBER_EPOCH (version 7+) + // - GROUP_ID_NOT_FOUND (version 9+) + // - STALE_MEMBER_EPOCH (version 9+) + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "3+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Topics", "type": "[]OffsetCommitResponseTopic", "versions": "0+", + "about": "The responses for each topic.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]OffsetCommitResponsePartition", "versions": "0+", + "about": "The responses for each partition in the topic.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." } + ]} + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/OffsetFetchRequest.json b/kafka/protocol/new/schemas/resources/OffsetFetchRequest.json new file mode 100644 index 000000000..88f5b568d --- /dev/null +++ b/kafka/protocol/new/schemas/resources/OffsetFetchRequest.json @@ -0,0 +1,72 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 9, + "type": "request", + "listeners": ["broker"], + "name": "OffsetFetchRequest", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // + // In version 0, the request read offsets from ZK. + // + // Starting in version 1, the broker supports fetching offsets from the internal __consumer_offsets topic. + // + // Starting in version 2, the request can contain a null topics array to indicate that offsets + // for all topics should be fetched. It also returns a top level error code + // for group or coordinator level errors. + // + // Version 3, 4, and 5 are the same as version 2. + // + // Version 6 is the first flexible version. + // + // Version 7 is adding the require stable flag. + // + // Version 8 is adding support for fetching offsets for multiple groups at a time. + // + // Version 9 is the first version that can be used with the new consumer group protocol (KIP-848). It adds + // the MemberId and MemberEpoch fields. Those are filled in and validated when the new consumer protocol is used. + "validVersions": "1-9", + "flexibleVersions": "6+", + "fields": [ + { "name": "GroupId", "type": "string", "versions": "0-7", "entityType": "groupId", + "about": "The group to fetch offsets for." }, + { "name": "Topics", "type": "[]OffsetFetchRequestTopic", "versions": "0-7", "nullableVersions": "2-7", + "about": "Each topic we would like to fetch offsets for, or null to fetch offsets for all topics.", "fields": [ + { "name": "Name", "type": "string", "versions": "0-7", "entityType": "topicName", + "about": "The topic name."}, + { "name": "PartitionIndexes", "type": "[]int32", "versions": "0-7", + "about": "The partition indexes we would like to fetch offsets for." } + ]}, + { "name": "Groups", "type": "[]OffsetFetchRequestGroup", "versions": "8+", + "about": "Each group we would like to fetch offsets for.", "fields": [ + { "name": "GroupId", "type": "string", "versions": "8+", "entityType": "groupId", + "about": "The group ID."}, + { "name": "MemberId", "type": "string", "versions": "9+", "nullableVersions": "9+", "default": "null", "ignorable": true, + "about": "The member id." }, + { "name": "MemberEpoch", "type": "int32", "versions": "9+", "default": "-1", "ignorable": true, + "about": "The member epoch if using the new consumer protocol (KIP-848)." }, + { "name": "Topics", "type": "[]OffsetFetchRequestTopics", "versions": "8+", "nullableVersions": "8+", + "about": "Each topic we would like to fetch offsets for, or null to fetch offsets for all topics.", "fields": [ + { "name": "Name", "type": "string", "versions": "8+", "entityType": "topicName", + "about": "The topic name."}, + { "name": "PartitionIndexes", "type": "[]int32", "versions": "8+", + "about": "The partition indexes we would like to fetch offsets for." } + ]} + ]}, + { "name": "RequireStable", "type": "bool", "versions": "7+", "default": "false", + "about": "Whether broker should hold on returning unstable offsets but set a retriable error code for the partitions." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/OffsetFetchResponse.json b/kafka/protocol/new/schemas/resources/OffsetFetchResponse.json new file mode 100644 index 000000000..9f0a5157c --- /dev/null +++ b/kafka/protocol/new/schemas/resources/OffsetFetchResponse.json @@ -0,0 +1,101 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 9, + "type": "response", + "name": "OffsetFetchResponse", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // + // Version 1 is the same as version 0. + // + // Version 2 adds a top-level error code. + // + // Version 3 adds the throttle time. + // + // Starting in version 4, on quota violation, brokers send out responses before throttling. + // + // Version 5 adds the leader epoch to the committed offset. + // + // Version 6 is the first flexible version. + // + // Version 7 adds pending offset commit as new error response on partition level. + // + // Version 8 is adding support for fetching offsets for multiple groups + // + // Version 9 is the first version that can be used with the new consumer group protocol (KIP-848). The response is + // the same as version 8 but can return STALE_MEMBER_EPOCH and UNKNOWN_MEMBER_ID errors when the new consumer group + // protocol is used. + "validVersions": "1-9", + "flexibleVersions": "6+", + // Supported errors: + // - GROUP_AUTHORIZATION_FAILED (version 0+) + // - NOT_COORDINATOR (version 0+) + // - COORDINATOR_NOT_AVAILABLE (version 0+) + // - COORDINATOR_LOAD_IN_PROGRESS (version 0+) + // - GROUP_ID_NOT_FOUND (version 0+) + // - UNSTABLE_OFFSET_COMMIT (version 7+) + // - UNKNOWN_MEMBER_ID (version 9+) + // - STALE_MEMBER_EPOCH (version 9+) + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "3+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Topics", "type": "[]OffsetFetchResponseTopic", "versions": "0-7", + "about": "The responses per topic.", "fields": [ + { "name": "Name", "type": "string", "versions": "0-7", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]OffsetFetchResponsePartition", "versions": "0-7", + "about": "The responses per partition.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0-7", + "about": "The partition index." }, + { "name": "CommittedOffset", "type": "int64", "versions": "0-7", + "about": "The committed message offset." }, + { "name": "CommittedLeaderEpoch", "type": "int32", "versions": "5-7", "default": "-1", + "ignorable": true, "about": "The leader epoch." }, + { "name": "Metadata", "type": "string", "versions": "0-7", "nullableVersions": "0-7", + "about": "The partition metadata." }, + { "name": "ErrorCode", "type": "int16", "versions": "0-7", + "about": "The error code, or 0 if there was no error." } + ]} + ]}, + { "name": "ErrorCode", "type": "int16", "versions": "2-7", "default": "0", "ignorable": true, + "about": "The top-level error code, or 0 if there was no error." }, + { "name": "Groups", "type": "[]OffsetFetchResponseGroup", "versions": "8+", + "about": "The responses per group id.", "fields": [ + { "name": "GroupId", "type": "string", "versions": "8+", "entityType": "groupId", + "about": "The group ID." }, + { "name": "Topics", "type": "[]OffsetFetchResponseTopics", "versions": "8+", + "about": "The responses per topic.", "fields": [ + { "name": "Name", "type": "string", "versions": "8+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]OffsetFetchResponsePartitions", "versions": "8+", + "about": "The responses per partition.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "8+", + "about": "The partition index." }, + { "name": "CommittedOffset", "type": "int64", "versions": "8+", + "about": "The committed message offset." }, + { "name": "CommittedLeaderEpoch", "type": "int32", "versions": "8+", "default": "-1", + "ignorable": true, "about": "The leader epoch." }, + { "name": "Metadata", "type": "string", "versions": "8+", "nullableVersions": "8+", + "about": "The partition metadata." }, + { "name": "ErrorCode", "type": "int16", "versions": "8+", + "about": "The partition-level error code, or 0 if there was no error." } + ]} + ]}, + { "name": "ErrorCode", "type": "int16", "versions": "8+", "default": "0", + "about": "The group-level error code, or 0 if there was no error." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/OffsetForLeaderEpochRequest.json b/kafka/protocol/new/schemas/resources/OffsetForLeaderEpochRequest.json new file mode 100644 index 000000000..dd559bc87 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/OffsetForLeaderEpochRequest.json @@ -0,0 +1,52 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 23, + "type": "request", + "listeners": ["broker"], + "name": "OffsetForLeaderEpochRequest", + // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline. + // + // Version 1 is the same as version 0. + // + // Version 2 adds the current leader epoch to support fencing. + // + // Version 3 adds ReplicaId (the default is -2 which conventionally represents a + // "debug" consumer which is allowed to see offsets beyond the high watermark). + // Followers will use this replicaId when using an older version of the protocol. + // + // Version 4 enables flexible versions. + "validVersions": "2-4", + "flexibleVersions": "4+", + "fields": [ + { "name": "ReplicaId", "type": "int32", "versions": "3+", "default": -2, "ignorable": true, "entityType": "brokerId", + "about": "The broker ID of the follower, of -1 if this request is from a consumer." }, + { "name": "Topics", "type": "[]OffsetForLeaderTopic", "versions": "0+", + "about": "Each topic to get offsets for.", "fields": [ + { "name": "Topic", "type": "string", "versions": "0+", "entityType": "topicName", + "mapKey": true, "about": "The topic name." }, + { "name": "Partitions", "type": "[]OffsetForLeaderPartition", "versions": "0+", + "about": "Each partition to get offsets for.", "fields": [ + { "name": "Partition", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "CurrentLeaderEpoch", "type": "int32", "versions": "2+", "default": "-1", "ignorable": true, + "about": "An epoch used to fence consumers/replicas with old metadata. If the epoch provided by the client is larger than the current epoch known to the broker, then the UNKNOWN_LEADER_EPOCH error code will be returned. If the provided epoch is smaller, then the FENCED_LEADER_EPOCH error code will be returned." }, + { "name": "LeaderEpoch", "type": "int32", "versions": "0+", + "about": "The epoch to look up an offset for." } + ]} + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/OffsetForLeaderEpochResponse.json b/kafka/protocol/new/schemas/resources/OffsetForLeaderEpochResponse.json new file mode 100644 index 000000000..f82aa09b7 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/OffsetForLeaderEpochResponse.json @@ -0,0 +1,51 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 23, + "type": "response", + "name": "OffsetForLeaderEpochResponse", + // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline. + // + // Version 1 added the leader epoch to the response. + // + // Version 2 added the throttle time. + // + // Version 3 is the same as version 2. + // + // Version 4 enables flexible versions. + "validVersions": "2-4", + "flexibleVersions": "4+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Topics", "type": "[]OffsetForLeaderTopicResult", "versions": "0+", + "about": "Each topic we fetched offsets for.", "fields": [ + { "name": "Topic", "type": "string", "versions": "0+", "entityType": "topicName", + "mapKey": true, "about": "The topic name." }, + { "name": "Partitions", "type": "[]EpochEndOffset", "versions": "0+", + "about": "Each partition in the topic we fetched offsets for.", "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code 0, or if there was no error." }, + { "name": "Partition", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "LeaderEpoch", "type": "int32", "versions": "1+", "default": "-1", "ignorable": true, + "about": "The leader epoch of the partition." }, + { "name": "EndOffset", "type": "int64", "versions": "0+", "default": "-1", + "about": "The end offset of the epoch." } + ]} + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/ProduceRequest.json b/kafka/protocol/new/schemas/resources/ProduceRequest.json new file mode 100644 index 000000000..0bb29f923 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/ProduceRequest.json @@ -0,0 +1,71 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 0, + "type": "request", + "listeners": ["broker"], + "name": "ProduceRequest", + // Versions 0-2 were removed in Apache Kafka 4.0, version 3 is the new baseline. Due to a bug in librdkafka, + // these versions have to be included in the api versions response (see KAFKA-18659), but are rejected otherwise. + // See `ApiKeys.PRODUCE_API_VERSIONS_RESPONSE_MIN_VERSION` for more details. + // + // Version 1 and 2 are the same as version 0. + // + // Version 3 adds the transactional ID, which is used for authorization when attempting to write + // transactional data. Version 3 also adds support for Kafka Message Format v2. + // + // Version 4 is the same as version 3, but the requester must be prepared to handle a + // KAFKA_STORAGE_ERROR. + // + // Version 5 and 6 are the same as version 3. + // + // Starting in version 7, records can be produced using ZStandard compression. See KIP-110. + // + // Starting in Version 8, response has RecordErrors and ErrorMessage. See KIP-467. + // + // Version 9 enables flexible versions. + // + // Version 10 is the same as version 9 (KIP-951). + // + // Version 11 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + // + // Version 12 is the same as version 11 (KIP-890). Note when produce requests are used in transaction, if + // transaction V2 (KIP_890 part 2) is enabled, the produce request will also include the function for a + // AddPartitionsToTxn call. If V2 is disabled, the client can't use produce request version higher than 11 within + // a transaction. + "validVersions": "3-12", + "flexibleVersions": "9+", + "fields": [ + { "name": "TransactionalId", "type": "string", "versions": "3+", "nullableVersions": "3+", "default": "null", "entityType": "transactionalId", + "about": "The transactional ID, or null if the producer is not transactional." }, + { "name": "Acks", "type": "int16", "versions": "0+", + "about": "The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR." }, + { "name": "TimeoutMs", "type": "int32", "versions": "0+", + "about": "The timeout to await a response in milliseconds." }, + { "name": "TopicData", "type": "[]TopicProduceData", "versions": "0+", + "about": "Each topic to produce to.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", "mapKey": true, + "about": "The topic name." }, + { "name": "PartitionData", "type": "[]PartitionProduceData", "versions": "0+", + "about": "Each partition to produce to.", "fields": [ + { "name": "Index", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "Records", "type": "records", "versions": "0+", "nullableVersions": "0+", + "about": "The record data to be produced." } + ]} + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/ProduceResponse.json b/kafka/protocol/new/schemas/resources/ProduceResponse.json new file mode 100644 index 000000000..fafcd8640 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/ProduceResponse.json @@ -0,0 +1,94 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 0, + "type": "response", + "name": "ProduceResponse", + // Versions 0-2 were removed in Apache Kafka 4.0, version 3 is the new baseline. Due to a bug in librdkafka, + // these versions have to be included in the api versions response (see KAFKA-18659), but are rejected otherwise. + // See `ApiKeys.PRODUCE_API_VERSIONS_RESPONSE_MIN_VERSION` for more details. + // + // Version 1 added the throttle time. + // Version 2 added the log append time. + // + // Version 3 is the same as version 2. + // + // Version 4 added KAFKA_STORAGE_ERROR as a possible error code. + // + // Version 5 added LogStartOffset to filter out spurious OutOfOrderSequenceExceptions on the client. + // + // Version 8 added RecordErrors and ErrorMessage to include information about + // records that cause the whole batch to be dropped. See KIP-467 for details. + // + // Version 9 enables flexible versions. + // + // Version 10 adds 'CurrentLeader' and 'NodeEndpoints' as tagged fields (KIP-951) + // + // Version 11 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + // + // Version 12 is the same as version 10 (KIP-890). + "validVersions": "3-12", + "flexibleVersions": "9+", + "fields": [ + { "name": "Responses", "type": "[]TopicProduceResponse", "versions": "0+", + "about": "Each produce response.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", "mapKey": true, + "about": "The topic name." }, + { "name": "PartitionResponses", "type": "[]PartitionProduceResponse", "versions": "0+", + "about": "Each partition that we produced to within the topic.", "fields": [ + { "name": "Index", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." }, + { "name": "BaseOffset", "type": "int64", "versions": "0+", + "about": "The base offset." }, + { "name": "LogAppendTimeMs", "type": "int64", "versions": "2+", "default": "-1", "ignorable": true, + "about": "The timestamp returned by broker after appending the messages. If CreateTime is used for the topic, the timestamp will be -1. If LogAppendTime is used for the topic, the timestamp will be the broker local time when the messages are appended." }, + { "name": "LogStartOffset", "type": "int64", "versions": "5+", "default": "-1", "ignorable": true, + "about": "The log start offset." }, + { "name": "RecordErrors", "type": "[]BatchIndexAndErrorMessage", "versions": "8+", "ignorable": true, + "about": "The batch indices of records that caused the batch to be dropped.", "fields": [ + { "name": "BatchIndex", "type": "int32", "versions": "8+", + "about": "The batch index of the record that caused the batch to be dropped." }, + { "name": "BatchIndexErrorMessage", "type": "string", "default": "null", "versions": "8+", "nullableVersions": "8+", + "about": "The error message of the record that caused the batch to be dropped."} + ]}, + { "name": "ErrorMessage", "type": "string", "default": "null", "versions": "8+", "nullableVersions": "8+", "ignorable": true, + "about": "The global error message summarizing the common root cause of the records that caused the batch to be dropped."}, + { "name": "CurrentLeader", "type": "LeaderIdAndEpoch", "versions": "10+", "taggedVersions": "10+", "tag": 0, + "about": "The leader broker that the producer should use for future requests.", "fields": [ + { "name": "LeaderId", "type": "int32", "versions": "10+", "default": "-1", "entityType": "brokerId", + "about": "The ID of the current leader or -1 if the leader is unknown."}, + { "name": "LeaderEpoch", "type": "int32", "versions": "10+", "default": "-1", + "about": "The latest known leader epoch."} + ]} + ]} + ]}, + { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, "default": "0", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "NodeEndpoints", "type": "[]NodeEndpoint", "versions": "10+", "taggedVersions": "10+", "tag": 0, + "about": "Endpoints for all current-leaders enumerated in PartitionProduceResponses, with errors NOT_LEADER_OR_FOLLOWER.", "fields": [ + { "name": "NodeId", "type": "int32", "versions": "10+", + "mapKey": true, "entityType": "brokerId", "about": "The ID of the associated node."}, + { "name": "Host", "type": "string", "versions": "10+", + "about": "The node's hostname." }, + { "name": "Port", "type": "int32", "versions": "10+", + "about": "The node's port." }, + { "name": "Rack", "type": "string", "versions": "10+", "nullableVersions": "10+", "default": "null", + "about": "The rack of the node, or null if it has not been assigned to a rack." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/RequestHeader.json b/kafka/protocol/new/schemas/resources/RequestHeader.json new file mode 100644 index 000000000..50589768f --- /dev/null +++ b/kafka/protocol/new/schemas/resources/RequestHeader.json @@ -0,0 +1,44 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "type": "header", + "name": "RequestHeader", + // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. + // + // Version 0 of the RequestHeader is only used by v0 of ControlledShutdownRequest. + // + // Version 1 is the first version with ClientId. + // + // Version 2 is the first flexible version. + "validVersions": "1-2", + "flexibleVersions": "2+", + "fields": [ + { "name": "RequestApiKey", "type": "int16", "versions": "0+", + "about": "The API key of this request." }, + { "name": "RequestApiVersion", "type": "int16", "versions": "0+", + "about": "The API version of this request." }, + { "name": "CorrelationId", "type": "int32", "versions": "0+", + "about": "The correlation ID of this request." }, + + // The ClientId string must be serialized with the old-style two-byte length prefix. + // The reason is that older brokers must be able to read the request header for any + // ApiVersionsRequest, even if it is from a newer version. + // Since the client is sending the ApiVersionsRequest in order to discover what + // versions are supported, the client does not know the best version to use. + { "name": "ClientId", "type": "string", "versions": "1+", "nullableVersions": "1+", "flexibleVersions": "none", + "about": "The client ID string." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/ResponseHeader.json b/kafka/protocol/new/schemas/resources/ResponseHeader.json new file mode 100644 index 000000000..773673601 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/ResponseHeader.json @@ -0,0 +1,26 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "type": "header", + "name": "ResponseHeader", + // Version 1 is the first flexible version. + "validVersions": "0-1", + "flexibleVersions": "1+", + "fields": [ + { "name": "CorrelationId", "type": "int32", "versions": "0+", + "about": "The correlation ID of this response." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/SaslAuthenticateRequest.json b/kafka/protocol/new/schemas/resources/SaslAuthenticateRequest.json new file mode 100644 index 000000000..cdb4247b8 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/SaslAuthenticateRequest.json @@ -0,0 +1,29 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 36, + "type": "request", + "listeners": ["broker", "controller"], + "name": "SaslAuthenticateRequest", + // Version 1 is the same as version 0. + // Version 2 adds flexible version support + "validVersions": "0-2", + "flexibleVersions": "2+", + "fields": [ + { "name": "AuthBytes", "type": "bytes", "versions": "0+", + "about": "The SASL authentication bytes from the client, as defined by the SASL mechanism." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/SaslAuthenticateResponse.json b/kafka/protocol/new/schemas/resources/SaslAuthenticateResponse.json new file mode 100644 index 000000000..edf383b9c --- /dev/null +++ b/kafka/protocol/new/schemas/resources/SaslAuthenticateResponse.json @@ -0,0 +1,34 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 36, + "type": "response", + "name": "SaslAuthenticateResponse", + // Version 1 adds the session lifetime. + // Version 2 adds flexible version support + "validVersions": "0-2", + "flexibleVersions": "2+", + "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." }, + { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "The error message, or null if there was no error." }, + { "name": "AuthBytes", "type": "bytes", "versions": "0+", + "about": "The SASL authentication bytes from the server, as defined by the SASL mechanism." }, + { "name": "SessionLifetimeMs", "type": "int64", "versions": "1+", "default": "0", "ignorable": true, + "about": "Number of milliseconds after which only re-authentication over the existing connection to create a new session can occur." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/SaslHandshakeRequest.json b/kafka/protocol/new/schemas/resources/SaslHandshakeRequest.json new file mode 100644 index 000000000..d2189d826 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/SaslHandshakeRequest.json @@ -0,0 +1,31 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 17, + "type": "request", + "listeners": ["broker", "controller"], + "name": "SaslHandshakeRequest", + // Version 1 supports SASL_AUTHENTICATE. + // NOTE: Version cannot be easily bumped due to incorrect + // client negotiation for clients <= 2.4. + // See https://issues.apache.org/jira/browse/KAFKA-9577 + "validVersions": "0-1", + "flexibleVersions": "none", + "fields": [ + { "name": "Mechanism", "type": "string", "versions": "0+", + "about": "The SASL mechanism chosen by the client." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/SaslHandshakeResponse.json b/kafka/protocol/new/schemas/resources/SaslHandshakeResponse.json new file mode 100644 index 000000000..a1567c669 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/SaslHandshakeResponse.json @@ -0,0 +1,32 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 17, + "type": "response", + "name": "SaslHandshakeResponse", + // Version 1 is the same as version 0. + // NOTE: Version cannot be easily bumped due to incorrect + // client negotiation for clients <= 2.4. + // See https://issues.apache.org/jira/browse/KAFKA-9577 + "validVersions": "0-1", + "flexibleVersions": "none", + "fields": [ + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." }, + { "name": "Mechanisms", "type": "[]string", "versions": "0+", + "about": "The mechanisms enabled in the server." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/SyncGroupRequest.json b/kafka/protocol/new/schemas/resources/SyncGroupRequest.json new file mode 100644 index 000000000..1b53df277 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/SyncGroupRequest.json @@ -0,0 +1,56 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 14, + "type": "request", + "listeners": ["broker"], + "name": "SyncGroupRequest", + // Versions 1 and 2 are the same as version 0. + // + // Starting from version 3, we add a new field called groupInstanceId to indicate member identity across restarts. + // + // Version 4 is the first flexible version. + // + // Starting from version 5, the client sends the Protocol Type and the Protocol Name + // to the broker (KIP-559). The broker will reject the request if they are inconsistent + // with the Type and Name known by the broker. + "validVersions": "0-5", + "flexibleVersions": "4+", + "fields": [ + { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", + "about": "The unique group identifier." }, + { "name": "GenerationId", "type": "int32", "versions": "0+", + "about": "The generation of the group." }, + { "name": "MemberId", "type": "string", "versions": "0+", + "about": "The member ID assigned by the group." }, + { "name": "GroupInstanceId", "type": "string", "versions": "3+", + "nullableVersions": "3+", "default": "null", + "about": "The unique identifier of the consumer instance provided by end user." }, + { "name": "ProtocolType", "type": "string", "versions": "5+", + "nullableVersions": "5+", "default": "null", "ignorable": true, + "about": "The group protocol type." }, + { "name": "ProtocolName", "type": "string", "versions": "5+", + "nullableVersions": "5+", "default": "null", "ignorable": true, + "about": "The group protocol name." }, + { "name": "Assignments", "type": "[]SyncGroupRequestAssignment", "versions": "0+", + "about": "Each assignment.", "fields": [ + { "name": "MemberId", "type": "string", "versions": "0+", + "about": "The ID of the member to assign." }, + { "name": "Assignment", "type": "bytes", "versions": "0+", + "about": "The member assignment." } + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/SyncGroupResponse.json b/kafka/protocol/new/schemas/resources/SyncGroupResponse.json new file mode 100644 index 000000000..4aa17e0d7 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/SyncGroupResponse.json @@ -0,0 +1,46 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 14, + "type": "response", + "name": "SyncGroupResponse", + // Version 1 adds throttle time. + // + // Starting in version 2, on quota violation, brokers send out responses before throttling. + // + // Starting from version 3, syncGroupRequest supports a new field called groupInstanceId to indicate member identity across restarts. + // + // Version 4 is the first flexible version. + // + // Starting from version 5, the broker sends back the Protocol Type and the Protocol Name + // to the client (KIP-559). + "validVersions": "0-5", + "flexibleVersions": "4+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." }, + { "name": "ProtocolType", "type": "string", "versions": "5+", + "nullableVersions": "5+", "default": "null", "ignorable": true, + "about": "The group protocol type." }, + { "name": "ProtocolName", "type": "string", "versions": "5+", + "nullableVersions": "5+", "default": "null", "ignorable": true, + "about": "The group protocol name." }, + { "name": "Assignment", "type": "bytes", "versions": "0+", + "about": "The member assignment." } + ] +} diff --git a/kafka/protocol/new/schemas/resources/TxnOffsetCommitRequest.json b/kafka/protocol/new/schemas/resources/TxnOffsetCommitRequest.json new file mode 100644 index 000000000..59a1f05e0 --- /dev/null +++ b/kafka/protocol/new/schemas/resources/TxnOffsetCommitRequest.json @@ -0,0 +1,68 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 28, + "type": "request", + "listeners": ["broker"], + "name": "TxnOffsetCommitRequest", + // Version 1 is the same as version 0. + // + // Version 2 adds the committed leader epoch. + // + // Version 3 adds the member.id, group.instance.id and generation.id. + // + // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + // + // Version 5 is the same as version 4 (KIP-890). Note when TxnOffsetCommit requests are used in transaction, if + // transaction V2 (KIP_890 part 2) is enabled, the TxnOffsetCommit request will also include the function for a + // AddOffsetsToTxn call. If V2 is disabled, the client can't use TxnOffsetCommit request version higher than 4 within + // a transaction. + "validVersions": "0-5", + "flexibleVersions": "3+", + "fields": [ + { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId", + "about": "The ID of the transaction." }, + { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", + "about": "The ID of the group." }, + { "name": "ProducerId", "type": "int64", "versions": "0+", "entityType": "producerId", + "about": "The current producer ID in use by the transactional ID." }, + { "name": "ProducerEpoch", "type": "int16", "versions": "0+", + "about": "The current epoch associated with the producer ID." }, + { "name": "GenerationId", "type": "int32", "versions": "3+", "default": "-1", + "about": "The generation of the consumer." }, + { "name": "MemberId", "type": "string", "versions": "3+", "default": "", + "about": "The member ID assigned by the group coordinator." }, + { "name": "GroupInstanceId", "type": "string", "versions": "3+", + "nullableVersions": "3+", "default": "null", + "about": "The unique identifier of the consumer instance provided by end user." }, + { "name": "Topics", "type" : "[]TxnOffsetCommitRequestTopic", "versions": "0+", + "about": "Each topic that we want to commit offsets for.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]TxnOffsetCommitRequestPartition", "versions": "0+", + "about": "The partitions inside the topic that we want to commit offsets for.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The index of the partition within the topic." }, + { "name": "CommittedOffset", "type": "int64", "versions": "0+", + "about": "The message offset to be committed." }, + { "name": "CommittedLeaderEpoch", "type": "int32", "versions": "2+", "default": "-1", "ignorable": true, + "about": "The leader epoch of the last consumed record." }, + { "name": "CommittedMetadata", "type": "string", "versions": "0+", "nullableVersions": "0+", + "about": "Any associated metadata the client wants to keep." } + ]} + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/TxnOffsetCommitResponse.json b/kafka/protocol/new/schemas/resources/TxnOffsetCommitResponse.json new file mode 100644 index 000000000..9769ed2aa --- /dev/null +++ b/kafka/protocol/new/schemas/resources/TxnOffsetCommitResponse.json @@ -0,0 +1,47 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "apiKey": 28, + "type": "response", + "name": "TxnOffsetCommitResponse", + // Starting in version 1, on quota violation, brokers send out responses before throttling. + // + // Version 2 is the same as version 1. + // + // Version 3 adds illegal generation, fenced instance id, and unknown member id errors. + // + // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). + // + // Version 5 is the same with version 3 (KIP-890). + "validVersions": "0-5", + "flexibleVersions": "3+", + "fields": [ + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", + "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, + { "name": "Topics", "type": "[]TxnOffsetCommitResponseTopic", "versions": "0+", + "about": "The responses for each topic.", "fields": [ + { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + "about": "The topic name." }, + { "name": "Partitions", "type": "[]TxnOffsetCommitResponsePartition", "versions": "0+", + "about": "The responses for each partition in the topic.", "fields": [ + { "name": "PartitionIndex", "type": "int32", "versions": "0+", + "about": "The partition index." }, + { "name": "ErrorCode", "type": "int16", "versions": "0+", + "about": "The error code, or 0 if there was no error." } + ]} + ]} + ] +} diff --git a/kafka/protocol/new/schemas/resources/__init__.py b/kafka/protocol/new/schemas/resources/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/pyproject.toml b/pyproject.toml index 312c45656..14a845598 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -42,6 +42,9 @@ benchmarks = ["pyperf"] include-package-data = false license-files = [] # workaround for https://github.com/pypa/setuptools/issues/4759 +[tool.setuptools.package-data] +kafka = ["protocol/new/schemas/resources/*.json"] + [tool.setuptools.packages.find] exclude = ["test"] namespaces = false diff --git a/test/protocol/new/admin/test_new_admin.py b/test/protocol/new/admin/test_new_admin.py new file mode 100644 index 000000000..15794da6b --- /dev/null +++ b/test/protocol/new/admin/test_new_admin.py @@ -0,0 +1,196 @@ +import uuid + +import pytest + +from kafka.protocol.new.admin import ( + CreateTopicsRequest, CreateTopicsResponse, + DeleteTopicsRequest, DeleteTopicsResponse, + CreatePartitionsRequest, CreatePartitionsResponse, + DescribeGroupsRequest, DescribeGroupsResponse, + ListGroupsRequest, ListGroupsResponse, + DeleteGroupsRequest, DeleteGroupsResponse, + DescribeClusterRequest, DescribeClusterResponse, + DescribeConfigsRequest, DescribeConfigsResponse, + AlterConfigsRequest, AlterConfigsResponse, + CreateAclsRequest, CreateAclsResponse, + DeleteAclsRequest, DeleteAclsResponse, + DescribeAclsRequest, DescribeAclsResponse, +) + + +@pytest.mark.parametrize("version", range(CreateTopicsRequest.min_version, CreateTopicsRequest.max_version + 1)) +def test_create_topics_request_roundtrip(version): + Topic = CreateTopicsRequest.CreatableTopic + topics = [ + Topic( + name="test-topic", + num_partitions=1, + replication_factor=1, + assignments=[], + configs=[] + ) + ] + data = CreateTopicsRequest( + topics=topics, + timeout_ms=10000, + validate_only=False + ) + encoded = CreateTopicsRequest.encode(data, version=version) + decoded = CreateTopicsRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(DeleteTopicsRequest.min_version, DeleteTopicsRequest.max_version + 1)) +def test_delete_topics_request_roundtrip(version): + topic_names = ["topic-1", "topic-2"] + + Topic = DeleteTopicsRequest.DeleteTopicState + topics = [] + if version >= 6: + for t_name in topic_names: + topics.append(Topic(name=t_name, topic_id=uuid.uuid4())) + + data = DeleteTopicsRequest( + topic_names=topic_names if version < 6 else [], + timeout_ms=10000, + topics=topics + ) + encoded = DeleteTopicsRequest.encode(data, version=version) + decoded = DeleteTopicsRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(DescribeGroupsRequest.min_version, DescribeGroupsRequest.max_version + 1)) +def test_describe_groups_request_roundtrip(version): + data = DescribeGroupsRequest( + groups=["group-1"], + include_authorized_operations=True if version >= 3 else False + ) + encoded = DescribeGroupsRequest.encode(data, version=version) + decoded = DescribeGroupsRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(ListGroupsRequest.min_version, ListGroupsRequest.max_version + 1)) +def test_list_groups_request_roundtrip(version): + data = ListGroupsRequest( + states_filter=["Stable"] if version >= 4 else [] + ) + encoded = ListGroupsRequest.encode(data, version=version) + decoded = ListGroupsRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(DescribeClusterRequest.min_version, DescribeClusterRequest.max_version + 1)) +def test_describe_cluster_request_roundtrip(version): + data = DescribeClusterRequest( + include_cluster_authorized_operations=True, + endpoint_type=1 if version >= 1 else 1, + include_fenced_brokers=False if version >= 2 else False + ) + encoded = DescribeClusterRequest.encode(data, version=version) + decoded = DescribeClusterRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(DescribeConfigsRequest.min_version, DescribeConfigsRequest.max_version + 1)) +def test_describe_configs_request_roundtrip(version): + Resource = DescribeConfigsRequest.DescribeConfigsResource + resources = [ + Resource( + resource_type=2, # TOPIC + resource_name="test-topic", + configuration_keys=["cleanup.policy"] + ) + ] + data = DescribeConfigsRequest( + resources=resources, + include_synonyms=True if version >= 1 else False, + include_documentation=True if version >= 3 else False + ) + encoded = DescribeConfigsRequest.encode(data, version=version) + decoded = DescribeConfigsRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(CreateAclsRequest.min_version, CreateAclsRequest.max_version + 1)) +def test_create_acls_request_roundtrip(version): + Creation = CreateAclsRequest.AclCreation + creations = [ + Creation( + resource_type=2, + resource_name="test-topic", + resource_pattern_type=3, + principal="User:alice", + host="*", + operation=3, + permission_type=3 + ) + ] + data = CreateAclsRequest( + creations=creations + ) + encoded = CreateAclsRequest.encode(data, version=version) + decoded = CreateAclsRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(CreatePartitionsRequest.min_version, CreatePartitionsRequest.max_version + 1)) +def test_create_partitions_request_roundtrip(version): + TopicPartition = CreatePartitionsRequest.CreatePartitionsTopic + Assignment = TopicPartition.CreatePartitionsAssignment + topic_partitions = [ + TopicPartition( + name="test-topic", + count=2, + assignments=[ + Assignment(broker_ids=[1, 2]) + ] + ) + ] + data = CreatePartitionsRequest( + topics=topic_partitions, + timeout_ms=10000, + validate_only=False + ) + encoded = CreatePartitionsRequest.encode(data, version=version) + decoded = CreatePartitionsRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(DescribeAclsRequest.min_version, DescribeAclsRequest.max_version + 1)) +def test_describe_acls_request_roundtrip(version): + data = DescribeAclsRequest( + resource_type_filter=2, + resource_name_filter="test-topic", + pattern_type_filter=3 if version >= 1 else 3, + principal_filter="User:alice", + host_filter="*", + operation=3, + permission_type=3 + ) + encoded = DescribeAclsRequest.encode(data, version=version) + decoded = DescribeAclsRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(DeleteAclsRequest.min_version, DeleteAclsRequest.max_version + 1)) +def test_delete_acls_request_roundtrip(version): + Filter = DeleteAclsRequest.DeleteAclsFilter + filters = [ + Filter( + resource_type_filter=2, + resource_name_filter="test-topic", + pattern_type_filter=3 if version >= 1 else 3, + principal_filter="User:alice", + host_filter="*", + operation=3, + permission_type=3 + ) + ] + data = DeleteAclsRequest( + filters=filters + ) + encoded = DeleteAclsRequest.encode(data, version=version) + decoded = DeleteAclsRequest.decode(encoded, version=version) + assert decoded == data diff --git a/test/protocol/new/consumer/test_new_fetch.py b/test/protocol/new/consumer/test_new_fetch.py new file mode 100644 index 000000000..030dc9922 --- /dev/null +++ b/test/protocol/new/consumer/test_new_fetch.py @@ -0,0 +1,128 @@ +import binascii +import pytest +import uuid + +from kafka.protocol.new.consumer import FetchRequest, FetchResponse + + +def test_fetch_request_v15_hex(): + # Hex dump provided by user + expected_hex = "0001000f0000007b00096d792d636c69656e7400000001f400000001000003e800123456780000007f01010100" + expected_bytes = binascii.unhexlify(expected_hex) + + req = FetchRequest( + max_wait_ms=500, + min_bytes=1, + max_bytes=1000, + isolation_level=0, + session_id=0x12345678, + session_epoch=0x7f, + topics=[], + forgotten_topics_data=[], + rack_id="" + ) + + req.with_header(correlation_id=123, client_id="my-client") + encoded = req.encode(version=15, header=True) + + assert encoded == expected_bytes + + # Decoding check + decoded = FetchRequest.decode(encoded, version=15, header=True) + assert decoded.header.correlation_id == 123 + assert decoded.header.client_id == "my-client" + assert decoded.session_id == 0x12345678 + assert decoded.session_epoch == 0x7f + assert decoded.max_wait_ms == 500 + + +@pytest.mark.parametrize("version", range(FetchRequest.min_version, FetchRequest.max_version + 1)) +def test_fetch_request_roundtrip(version): + # Topic data needs to match the version's requirements (Topic vs TopicId) + topic_data = [] + if version < 13: + topic_data = [ + FetchRequest.FetchTopic( + topic="test-topic", + partitions=[ + FetchRequest.FetchTopic.FetchPartition( + partition=0, + fetch_offset=100, + partition_max_bytes=1024 + ) + ] + ) + ] + else: + topic_id = uuid.uuid4() + topic_data = [ + FetchRequest.FetchTopic( + topic_id=topic_id, + partitions=[ + FetchRequest.FetchTopic.FetchPartition( + partition=0, + fetch_offset=100, + partition_max_bytes=1024 + ) + ] + ) + ] + + data = FetchRequest( + replica_id=-1, + max_wait_ms=500, + min_bytes=1, + topics=topic_data + ) + + encoded = FetchRequest.encode(data, version=version) + decoded = FetchRequest.decode(encoded, version=version) + + assert decoded == data + + +@pytest.mark.parametrize("version", range(FetchResponse.min_version, FetchResponse.max_version + 1)) +def test_fetch_response_roundtrip(version): + # Mocking some response data + resp_topic_data = [] + if version < 13: + resp_topic_data = [ + FetchResponse.FetchableTopicResponse( + topic="test-topic", + partitions=[ + FetchResponse.FetchableTopicResponse.PartitionData( + partition_index=0, + error_code=0, + high_watermark=1000, + records=b"some records" + ) + ] + ) + ] + else: + topic_id = uuid.uuid4() + resp_topic_data = [ + FetchResponse.FetchableTopicResponse( + topic_id=topic_id, + partitions=[ + FetchResponse.FetchableTopicResponse.PartitionData( + partition_index=0, + error_code=0, + high_watermark=1000, + records=b"some records" + ) + ] + ) + ] + + data = FetchResponse( + throttle_time_ms=100 if version >= 1 else 0, + error_code=0, + session_id=12345 if version >= 7 else 0, + responses=resp_topic_data + ) + + encoded = FetchResponse.encode(data, version=version) + decoded = FetchResponse.decode(encoded, version=version) + + assert decoded == data diff --git a/test/protocol/new/consumer/test_new_group.py b/test/protocol/new/consumer/test_new_group.py new file mode 100644 index 000000000..e4aa909bb --- /dev/null +++ b/test/protocol/new/consumer/test_new_group.py @@ -0,0 +1,333 @@ +import pytest + +from kafka.protocol.new.metadata import FindCoordinatorRequest, FindCoordinatorResponse +from kafka.protocol.new.consumer import ( + JoinGroupRequest, JoinGroupResponse, + SyncGroupRequest, SyncGroupResponse, + LeaveGroupRequest, LeaveGroupResponse, + HeartbeatRequest, HeartbeatResponse, + OffsetFetchRequest, OffsetFetchResponse, + OffsetCommitRequest, OffsetCommitResponse, +) + + +@pytest.mark.parametrize("version", range(FindCoordinatorRequest.min_version, FindCoordinatorRequest.max_version + 1)) +def test_find_coordinator_request_roundtrip(version): + data = FindCoordinatorRequest( + key="test-group" if version < 4 else "", + key_type=0, + coordinator_keys=["test-group"] if version >= 4 else [] + ) + encoded = FindCoordinatorRequest.encode(data, version=version) + decoded = FindCoordinatorRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(FindCoordinatorResponse.min_version, FindCoordinatorResponse.max_version + 1)) +def test_find_coordinator_response_roundtrip(version): + coordinators = [ + FindCoordinatorResponse.Coordinator( + key="test-group", + node_id=1, + host="localhost", + port=9092, + error_code=0, + error_message=None + ) + ] + data = FindCoordinatorResponse( + throttle_time_ms=100 if version >= 2 else 0, + error_code=0, + error_message="", + node_id=1 if version < 4 else 0, + host="localhost" if version < 4 else "", + port=9092 if version < 4 else 0, + coordinators=coordinators if version >= 4 else [] + ) + encoded = FindCoordinatorResponse.encode(data, version=version) + decoded = FindCoordinatorResponse.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(JoinGroupRequest.min_version, JoinGroupRequest.max_version + 1)) +def test_join_group_request_roundtrip(version): + protocols = [ + JoinGroupRequest.JoinGroupRequestProtocol( + name="range", + metadata=b"protocol-metadata" + ) + ] + data = JoinGroupRequest( + group_id="test-group", + session_timeout_ms=30000, + rebalance_timeout_ms=60000 if version >= 1 else -1, + member_id="test-member", + group_instance_id=None, + protocol_type="consumer", + protocols=protocols, + reason="joining" if version >= 8 else None + ) + encoded = JoinGroupRequest.encode(data, version=version) + decoded = JoinGroupRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(JoinGroupResponse.min_version, JoinGroupResponse.max_version + 1)) +def test_join_group_response_roundtrip(version): + members = [ + JoinGroupResponse.JoinGroupResponseMember( + member_id="test-member", + group_instance_id=None, + metadata=b"member-metadata" + ) + ] + data = JoinGroupResponse( + throttle_time_ms=100 if version >= 3 else 0, + error_code=0, + generation_id=1, + protocol_type="consumer" if version >= 7 else None, + protocol_name="range", + leader="test-member", + member_id="test-member", + members=members + ) + encoded = JoinGroupResponse.encode(data, version=version) + decoded = JoinGroupResponse.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(SyncGroupRequest.min_version, SyncGroupRequest.max_version + 1)) +def test_sync_group_request_roundtrip(version): + assignments = [ + SyncGroupRequest.SyncGroupRequestAssignment( + member_id="test-member", + assignment=b"test-assignment" + ) + ] + data = SyncGroupRequest( + group_id="test-group", + generation_id=1, + member_id="test-member", + group_instance_id=None, + assignments=assignments if version < 5 else [], + protocol_type="consumer" if version >= 5 else None, + protocol_name="range" if version >= 5 else None + ) + encoded = SyncGroupRequest.encode(data, version=version) + decoded = SyncGroupRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(SyncGroupResponse.min_version, SyncGroupResponse.max_version + 1)) +def test_sync_group_response_roundtrip(version): + data = SyncGroupResponse( + throttle_time_ms=100 if version >= 1 else 0, + error_code=0, + protocol_type="consumer" if version >= 5 else None, + protocol_name="range" if version >= 5 else None, + assignment=b"test-assignment" + ) + encoded = SyncGroupResponse.encode(data, version=version) + decoded = SyncGroupResponse.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(HeartbeatRequest.min_version, HeartbeatRequest.max_version + 1)) +def test_heartbeat_request_roundtrip(version): + data = HeartbeatRequest( + group_id="test-group", + generation_id=1, + member_id="test-member", + group_instance_id=None + ) + encoded = HeartbeatRequest.encode(data, version=version) + decoded = HeartbeatRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(HeartbeatResponse.min_version, HeartbeatResponse.max_version + 1)) +def test_heartbeat_response_roundtrip(version): + data = HeartbeatResponse( + throttle_time_ms=100 if version >= 1 else 0, + error_code=0 + ) + encoded = HeartbeatResponse.encode(data, version=version) + decoded = HeartbeatResponse.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(LeaveGroupRequest.min_version, LeaveGroupRequest.max_version + 1)) +def test_leave_group_request_roundtrip(version): + members = [ + LeaveGroupRequest.MemberIdentity( + member_id="test-member", + group_instance_id=None, + reason="leaving" if version >= 5 else None + ) + ] + data = LeaveGroupRequest( + group_id="test-group", + member_id="test-member" if version < 3 else "", + members=members if version >= 3 else [] + ) + encoded = LeaveGroupRequest.encode(data, version=version) + decoded = LeaveGroupRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(LeaveGroupResponse.min_version, LeaveGroupResponse.max_version + 1)) +def test_leave_group_response_roundtrip(version): + members = [ + LeaveGroupResponse.MemberResponse( + member_id="test-member", + group_instance_id=None, + error_code=0 + ) + ] + data = LeaveGroupResponse( + throttle_time_ms=100 if version >= 1 else 0, + error_code=0, + members=members if version >= 3 else [] + ) + encoded = LeaveGroupResponse.encode(data, version=version) + decoded = LeaveGroupResponse.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(OffsetFetchRequest.min_version, OffsetFetchRequest.max_version + 1)) +def test_offset_fetch_request_roundtrip(version): + if version < 8: + topics = [ + OffsetFetchRequest.OffsetFetchRequestTopic( + name="topic-1", + partition_indexes=[0, 1] + ) + ] + data = OffsetFetchRequest( + group_id="test-group", + topics=topics, + require_stable=False if version >= 7 else False + ) + else: + groups = [ + OffsetFetchRequest.OffsetFetchRequestGroup( + group_id="test-group", + topics=[ + OffsetFetchRequest.OffsetFetchRequestGroup.OffsetFetchRequestTopics( + name="topic-1", + partition_indexes=[0, 1] + ) + ] + ) + ] + data = OffsetFetchRequest( + groups=groups, + require_stable=False + ) + + encoded = OffsetFetchRequest.encode(data, version=version) + decoded = OffsetFetchRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(OffsetFetchResponse.min_version, OffsetFetchResponse.max_version + 1)) +def test_offset_fetch_response_roundtrip(version): + if version < 8: + topics = [ + OffsetFetchResponse.OffsetFetchResponseTopic( + name="topic-1", + partitions=[ + OffsetFetchResponse.OffsetFetchResponseTopic.OffsetFetchResponsePartition( + partition_index=0, + committed_offset=100, + committed_leader_epoch=1 if version >= 5 else -1, + metadata="meta", + error_code=0 + ) + ] + ) + ] + data = OffsetFetchResponse( + throttle_time_ms=100 if version >= 3 else 0, + topics=topics, + error_code=0 if version >= 2 else 0 + ) + else: + groups = [ + OffsetFetchResponse.OffsetFetchResponseGroup( + group_id="test-group", + topics=[ + OffsetFetchResponse.OffsetFetchResponseGroup.OffsetFetchResponseTopics( + name="topic-1", + partitions=[ + OffsetFetchResponse.OffsetFetchResponseGroup.OffsetFetchResponseTopics.OffsetFetchResponsePartitions( + partition_index=0, + committed_offset=100, + committed_leader_epoch=1, + metadata="meta", + error_code=0 + ) + ] + ) + ], + error_code=0 + ) + ] + data = OffsetFetchResponse( + throttle_time_ms=100, + groups=groups + ) + + encoded = OffsetFetchResponse.encode(data, version=version) + decoded = OffsetFetchResponse.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(OffsetCommitRequest.min_version, OffsetCommitRequest.max_version + 1)) +def test_offset_commit_request_roundtrip(version): + topics = [ + OffsetCommitRequest.OffsetCommitRequestTopic( + name="topic-1", + partitions=[ + OffsetCommitRequest.OffsetCommitRequestTopic.OffsetCommitRequestPartition( + partition_index=0, + committed_offset=100, + committed_leader_epoch=1 if version >= 6 else -1, + committed_metadata="meta" + ) + ] + ) + ] + data = OffsetCommitRequest( + group_id="test-group", + generation_id_or_member_epoch=1 if version >= 1 else -1, + member_id="test-member" if version >= 1 else "", + group_instance_id=None, + retention_time_ms=5000 if 2 <= version <= 4 else -1, + topics=topics + ) + encoded = OffsetCommitRequest.encode(data, version=version) + decoded = OffsetCommitRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(OffsetCommitResponse.min_version, OffsetCommitResponse.max_version + 1)) +def test_offset_commit_response_roundtrip(version): + topics = [ + OffsetCommitResponse.OffsetCommitResponseTopic( + name="topic-1", + partitions=[ + OffsetCommitResponse.OffsetCommitResponseTopic.OffsetCommitResponsePartition( + partition_index=0, + error_code=0 + ) + ] + ) + ] + data = OffsetCommitResponse( + throttle_time_ms=100 if version >= 3 else 0, + topics=topics + ) + encoded = OffsetCommitResponse.encode(data, version=version) + decoded = OffsetCommitResponse.decode(encoded, version=version) + assert decoded == data diff --git a/test/protocol/new/consumer/test_new_list_offsets.py b/test/protocol/new/consumer/test_new_list_offsets.py new file mode 100644 index 000000000..fdc50029d --- /dev/null +++ b/test/protocol/new/consumer/test_new_list_offsets.py @@ -0,0 +1,54 @@ +import pytest + +from kafka.protocol.new.consumer import ListOffsetsRequest, ListOffsetsResponse + + +@pytest.mark.parametrize("version", range(ListOffsetsRequest.min_version, ListOffsetsRequest.max_version + 1)) +def test_list_offsets_request_roundtrip(version): + topics = [ + ListOffsetsRequest.ListOffsetsTopic( + name="topic-1", + partitions=[ + ListOffsetsRequest.ListOffsetsTopic.ListOffsetsPartition( + partition_index=0, + current_leader_epoch=-1 if version >= 9 else -1, # default is -1 + timestamp=1000 + ) + ] + ) + ] + data = ListOffsetsRequest( + replica_id=-1, + isolation_level=0, + topics=topics + ) + + encoded = ListOffsetsRequest.encode(data, version=version) + decoded = ListOffsetsRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(ListOffsetsResponse.min_version, ListOffsetsResponse.max_version + 1)) +def test_list_offsets_response_roundtrip(version): + topics = [ + ListOffsetsResponse.ListOffsetsTopicResponse( + name="topic-1", + partitions=[ + ListOffsetsResponse.ListOffsetsTopicResponse.ListOffsetsPartitionResponse( + partition_index=0, + error_code=0, + timestamp=1000 if version >= 1 else -1, + offset=1000 if version >= 1 else -1, + leader_epoch=1 if version >= 4 else -1 + ) + ] + ) + ] + data = ListOffsetsResponse( + throttle_time_ms=10 if version >= 2 else 0, + topics=topics + ) + + encoded = ListOffsetsResponse.encode(data, version=version) + decoded = ListOffsetsResponse.decode(encoded, version=version) + assert decoded == data diff --git a/test/protocol/new/metadata/test_new_api_versions.py b/test/protocol/new/metadata/test_new_api_versions.py new file mode 100644 index 000000000..9aa33750c --- /dev/null +++ b/test/protocol/new/metadata/test_new_api_versions.py @@ -0,0 +1,78 @@ +import pytest + + +from kafka.protocol.new.api_header import ResponseHeader +from kafka.protocol.new.metadata import ApiVersionsRequest, ApiVersionsResponse + + +TEST_CASES = [ + ( + ApiVersionsRequest[0](), + b'\x00\x00\x00\x1f\x00\x12\x00\x00\x00\x00\x00\x01\x00\x15_internal_client_kYVL', + ), + + ( + ApiVersionsRequest[1](), + b'\x00\x00\x00\x1f\x00\x12\x00\x01\x00\x00\x00\x01\x00\x15_internal_client_kYVL', + ), + + ( + ApiVersionsRequest[2](), + b'\x00\x00\x00\x1f\x00\x12\x00\x02\x00\x00\x00\x01\x00\x15_internal_client_kYVL', + ), + + ( + ApiVersionsRequest[3](client_software_name='kafka-python', client_software_version='3.0.0'), + b'\x00\x00\x004\x00\x12\x00\x03\x00\x00\x00\x01\x00\x15_internal_client_kYVL\x00\rkafka-python\x063.0.0\x00', + ), + + ( + ApiVersionsRequest[4](client_software_name='kafka-python', client_software_version='3.0.0'), + b'\x00\x00\x004\x00\x12\x00\x04\x00\x00\x00\x01\x00\x15_internal_client_kYVL\x00\rkafka-python\x063.0.0\x00', + ), + + ( + ApiVersionsResponse[0](error_code=35, api_keys=[(18, 0, 3)]), + b'\x00\x00\x00\x10\x00\x00\x00\x01\x00#\x00\x00\x00\x01\x00\x12\x00\x00\x00\x03', + ), + + ( + ApiVersionsResponse[0](error_code=0, api_keys=[(0, 0, 8), (1, 0, 11), (2, 0, 5), (3, 0, 9), (4, 0, 4), (5, 0, 3), (6, 0, 6), (7, 0, 3), (8, 0, 8), (9, 0, 7), (10, 0, 3), (11, 0, 7), (12, 0, 4), (13, 0, 4), (14, 0, 5), (15, 0, 5), (16, 0, 4), (17, 0, 1), (18, 0, 3), (19, 0, 5), (20, 0, 4), (21, 0, 2), (22, 0, 3), (23, 0, 3), (24, 0, 1), (25, 0, 1), (26, 0, 1), (27, 0, 0), (28, 0, 3), (29, 0, 2), (30, 0, 2), (31, 0, 2), (32, 0, 3), (33, 0, 1), (34, 0, 1), (35, 0, 2), (36, 0, 2), (37, 0, 2), (38, 0, 2), (39, 0, 2), (40, 0, 2), (41, 0, 2), (42, 0, 2), (43, 0, 2), (44, 0, 1), (45, 0, 0), (46, 0, 0), (47, 0, 0), (48, 0, 0), (49, 0, 0)]), + + b'\x00\x00\x016\x00\x00\x00\x01\x00\x00\x00\x00\x002\x00\x00\x00\x00\x00\x08\x00\x01\x00\x00\x00\x0b\x00\x02\x00\x00\x00\x05\x00\x03\x00\x00\x00\t\x00\x04\x00\x00\x00\x04\x00\x05\x00\x00\x00\x03\x00\x06\x00\x00\x00\x06\x00\x07\x00\x00\x00\x03\x00\x08\x00\x00\x00\x08\x00\t\x00\x00\x00\x07\x00\n\x00\x00\x00\x03\x00\x0b\x00\x00\x00\x07\x00\x0c\x00\x00\x00\x04\x00\r\x00\x00\x00\x04\x00\x0e\x00\x00\x00\x05\x00\x0f\x00\x00\x00\x05\x00\x10\x00\x00\x00\x04\x00\x11\x00\x00\x00\x01\x00\x12\x00\x00\x00\x03\x00\x13\x00\x00\x00\x05\x00\x14\x00\x00\x00\x04\x00\x15\x00\x00\x00\x02\x00\x16\x00\x00\x00\x03\x00\x17\x00\x00\x00\x03\x00\x18\x00\x00\x00\x01\x00\x19\x00\x00\x00\x01\x00\x1a\x00\x00\x00\x01\x00\x1b\x00\x00\x00\x00\x00\x1c\x00\x00\x00\x03\x00\x1d\x00\x00\x00\x02\x00\x1e\x00\x00\x00\x02\x00\x1f\x00\x00\x00\x02\x00 \x00\x00\x00\x03\x00!\x00\x00\x00\x01\x00"\x00\x00\x00\x01\x00#\x00\x00\x00\x02\x00$\x00\x00\x00\x02\x00%\x00\x00\x00\x02\x00&\x00\x00\x00\x02\x00\'\x00\x00\x00\x02\x00(\x00\x00\x00\x02\x00)\x00\x00\x00\x02\x00*\x00\x00\x00\x02\x00+\x00\x00\x00\x02\x00,\x00\x00\x00\x01\x00-\x00\x00\x00\x00\x00.\x00\x00\x00\x00\x00/\x00\x00\x00\x00\x000\x00\x00\x00\x00\x001\x00\x00\x00\x00', + ), + + ( + ApiVersionsResponse[1](error_code=0, api_keys=[(0, 0, 8), (1, 0, 11), (2, 0, 5), (3, 0, 9), (4, 0, 4), (5, 0, 3), (6, 0, 6), (7, 0, 3), (8, 0, 8), (9, 0, 7), (10, 0, 3), (11, 0, 7), (12, 0, 4), (13, 0, 4), (14, 0, 5), (15, 0, 5), (16, 0, 4), (17, 0, 1), (18, 0, 3), (19, 0, 5), (20, 0, 4), (21, 0, 2), (22, 0, 3), (23, 0, 3), (24, 0, 1), (25, 0, 1), (26, 0, 1), (27, 0, 0), (28, 0, 3), (29, 0, 2), (30, 0, 2), (31, 0, 2), (32, 0, 3), (33, 0, 1), (34, 0, 1), (35, 0, 2), (36, 0, 2), (37, 0, 2), (38, 0, 2), (39, 0, 2), (40, 0, 2), (41, 0, 2), (42, 0, 2), (43, 0, 2), (44, 0, 1), (45, 0, 0), (46, 0, 0), (47, 0, 0), (48, 0, 0), (49, 0, 0)], throttle_time_ms=1), + + b'\x00\x00\x01:\x00\x00\x00\x01\x00\x00\x00\x00\x002\x00\x00\x00\x00\x00\x08\x00\x01\x00\x00\x00\x0b\x00\x02\x00\x00\x00\x05\x00\x03\x00\x00\x00\t\x00\x04\x00\x00\x00\x04\x00\x05\x00\x00\x00\x03\x00\x06\x00\x00\x00\x06\x00\x07\x00\x00\x00\x03\x00\x08\x00\x00\x00\x08\x00\t\x00\x00\x00\x07\x00\n\x00\x00\x00\x03\x00\x0b\x00\x00\x00\x07\x00\x0c\x00\x00\x00\x04\x00\r\x00\x00\x00\x04\x00\x0e\x00\x00\x00\x05\x00\x0f\x00\x00\x00\x05\x00\x10\x00\x00\x00\x04\x00\x11\x00\x00\x00\x01\x00\x12\x00\x00\x00\x03\x00\x13\x00\x00\x00\x05\x00\x14\x00\x00\x00\x04\x00\x15\x00\x00\x00\x02\x00\x16\x00\x00\x00\x03\x00\x17\x00\x00\x00\x03\x00\x18\x00\x00\x00\x01\x00\x19\x00\x00\x00\x01\x00\x1a\x00\x00\x00\x01\x00\x1b\x00\x00\x00\x00\x00\x1c\x00\x00\x00\x03\x00\x1d\x00\x00\x00\x02\x00\x1e\x00\x00\x00\x02\x00\x1f\x00\x00\x00\x02\x00 \x00\x00\x00\x03\x00!\x00\x00\x00\x01\x00"\x00\x00\x00\x01\x00#\x00\x00\x00\x02\x00$\x00\x00\x00\x02\x00%\x00\x00\x00\x02\x00&\x00\x00\x00\x02\x00\'\x00\x00\x00\x02\x00(\x00\x00\x00\x02\x00)\x00\x00\x00\x02\x00*\x00\x00\x00\x02\x00+\x00\x00\x00\x02\x00,\x00\x00\x00\x01\x00-\x00\x00\x00\x00\x00.\x00\x00\x00\x00\x00/\x00\x00\x00\x00\x000\x00\x00\x00\x00\x001\x00\x00\x00\x00\x00\x00\x00\x01', + ), + + ( + ApiVersionsResponse[2](error_code=0, api_keys=[(0, 0, 8), (1, 0, 11), (2, 0, 5), (3, 0, 9), (4, 0, 4), (5, 0, 3), (6, 0, 6), (7, 0, 3), (8, 0, 8), (9, 0, 7), (10, 0, 3), (11, 0, 7), (12, 0, 4), (13, 0, 4), (14, 0, 5), (15, 0, 5), (16, 0, 4), (17, 0, 1), (18, 0, 3), (19, 0, 5), (20, 0, 4), (21, 0, 2), (22, 0, 3), (23, 0, 3), (24, 0, 1), (25, 0, 1), (26, 0, 1), (27, 0, 0), (28, 0, 3), (29, 0, 2), (30, 0, 2), (31, 0, 2), (32, 0, 3), (33, 0, 1), (34, 0, 1), (35, 0, 2), (36, 0, 2), (37, 0, 2), (38, 0, 2), (39, 0, 2), (40, 0, 2), (41, 0, 2), (42, 0, 2), (43, 0, 2), (44, 0, 1), (45, 0, 0), (46, 0, 0), (47, 0, 0), (48, 0, 0), (49, 0, 0)], throttle_time_ms=1), + + b'\x00\x00\x01:\x00\x00\x00\x01\x00\x00\x00\x00\x002\x00\x00\x00\x00\x00\x08\x00\x01\x00\x00\x00\x0b\x00\x02\x00\x00\x00\x05\x00\x03\x00\x00\x00\t\x00\x04\x00\x00\x00\x04\x00\x05\x00\x00\x00\x03\x00\x06\x00\x00\x00\x06\x00\x07\x00\x00\x00\x03\x00\x08\x00\x00\x00\x08\x00\t\x00\x00\x00\x07\x00\n\x00\x00\x00\x03\x00\x0b\x00\x00\x00\x07\x00\x0c\x00\x00\x00\x04\x00\r\x00\x00\x00\x04\x00\x0e\x00\x00\x00\x05\x00\x0f\x00\x00\x00\x05\x00\x10\x00\x00\x00\x04\x00\x11\x00\x00\x00\x01\x00\x12\x00\x00\x00\x03\x00\x13\x00\x00\x00\x05\x00\x14\x00\x00\x00\x04\x00\x15\x00\x00\x00\x02\x00\x16\x00\x00\x00\x03\x00\x17\x00\x00\x00\x03\x00\x18\x00\x00\x00\x01\x00\x19\x00\x00\x00\x01\x00\x1a\x00\x00\x00\x01\x00\x1b\x00\x00\x00\x00\x00\x1c\x00\x00\x00\x03\x00\x1d\x00\x00\x00\x02\x00\x1e\x00\x00\x00\x02\x00\x1f\x00\x00\x00\x02\x00 \x00\x00\x00\x03\x00!\x00\x00\x00\x01\x00"\x00\x00\x00\x01\x00#\x00\x00\x00\x02\x00$\x00\x00\x00\x02\x00%\x00\x00\x00\x02\x00&\x00\x00\x00\x02\x00\'\x00\x00\x00\x02\x00(\x00\x00\x00\x02\x00)\x00\x00\x00\x02\x00*\x00\x00\x00\x02\x00+\x00\x00\x00\x02\x00,\x00\x00\x00\x01\x00-\x00\x00\x00\x00\x00.\x00\x00\x00\x00\x00/\x00\x00\x00\x00\x000\x00\x00\x00\x00\x001\x00\x00\x00\x00\x00\x00\x00\x01', + ), + + ( + ApiVersionsResponse[3](error_code=0, api_keys=[(0, 0, 8, {}), (1, 0, 11, {}), (2, 0, 5, {}), (3, 0, 9, {}), (4, 0, 4, {}), (5, 0, 3, {}), (6, 0, 6, {}), (7, 0, 3, {}), (8, 0, 8, {}), (9, 0, 7, {}), (10, 0, 3, {}), (11, 0, 7, {}), (12, 0, 4, {}), (13, 0, 4, {}), (14, 0, 5, {}), (15, 0, 5, {}), (16, 0, 4, {}), (17, 0, 1, {}), (18, 0, 3, {}), (19, 0, 5, {}), (20, 0, 4, {}), (21, 0, 2, {}), (22, 0, 3, {}), (23, 0, 3, {}), (24, 0, 1, {}), (25, 0, 1, {}), (26, 0, 1, {}), (27, 0, 0, {}), (28, 0, 3, {}), (29, 0, 2, {}), (30, 0, 2, {}), (31, 0, 2, {}), (32, 0, 3, {}), (33, 0, 1, {}), (34, 0, 1, {}), (35, 0, 2, {}), (36, 0, 2, {}), (37, 0, 2, {}), (38, 0, 2, {}), (39, 0, 2, {}), (40, 0, 2, {}), (41, 0, 2, {}), (42, 0, 2, {}), (43, 0, 2, {}), (44, 0, 1, {}), (45, 0, 0, {}), (46, 0, 0, {}), (47, 0, 0, {}), (48, 0, 0, {}), (49, 0, 0, {})], throttle_time_ms=0), + + b'\x00\x00\x01j\x00\x00\x00\x01\x00\x003\x00\x00\x00\x00\x00\x08\x00\x00\x01\x00\x00\x00\x0b\x00\x00\x02\x00\x00\x00\x05\x00\x00\x03\x00\x00\x00\t\x00\x00\x04\x00\x00\x00\x04\x00\x00\x05\x00\x00\x00\x03\x00\x00\x06\x00\x00\x00\x06\x00\x00\x07\x00\x00\x00\x03\x00\x00\x08\x00\x00\x00\x08\x00\x00\t\x00\x00\x00\x07\x00\x00\n\x00\x00\x00\x03\x00\x00\x0b\x00\x00\x00\x07\x00\x00\x0c\x00\x00\x00\x04\x00\x00\r\x00\x00\x00\x04\x00\x00\x0e\x00\x00\x00\x05\x00\x00\x0f\x00\x00\x00\x05\x00\x00\x10\x00\x00\x00\x04\x00\x00\x11\x00\x00\x00\x01\x00\x00\x12\x00\x00\x00\x03\x00\x00\x13\x00\x00\x00\x05\x00\x00\x14\x00\x00\x00\x04\x00\x00\x15\x00\x00\x00\x02\x00\x00\x16\x00\x00\x00\x03\x00\x00\x17\x00\x00\x00\x03\x00\x00\x18\x00\x00\x00\x01\x00\x00\x19\x00\x00\x00\x01\x00\x00\x1a\x00\x00\x00\x01\x00\x00\x1b\x00\x00\x00\x00\x00\x00\x1c\x00\x00\x00\x03\x00\x00\x1d\x00\x00\x00\x02\x00\x00\x1e\x00\x00\x00\x02\x00\x00\x1f\x00\x00\x00\x02\x00\x00 \x00\x00\x00\x03\x00\x00!\x00\x00\x00\x01\x00\x00"\x00\x00\x00\x01\x00\x00#\x00\x00\x00\x02\x00\x00$\x00\x00\x00\x02\x00\x00%\x00\x00\x00\x02\x00\x00&\x00\x00\x00\x02\x00\x00\'\x00\x00\x00\x02\x00\x00(\x00\x00\x00\x02\x00\x00)\x00\x00\x00\x02\x00\x00*\x00\x00\x00\x02\x00\x00+\x00\x00\x00\x02\x00\x00,\x00\x00\x00\x01\x00\x00-\x00\x00\x00\x00\x00\x00.\x00\x00\x00\x00\x00\x00/\x00\x00\x00\x00\x00\x000\x00\x00\x00\x00\x00\x001\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00', + ), +] + + +@pytest.mark.parametrize('msg, encoded', TEST_CASES) +def test_parse(msg, encoded): + msg.with_header(correlation_id=1, client_id='_internal_client_kYVL') + assert msg.encode(header=True, framed=True) == encoded + assert msg.decode(encoded, header=True, framed=True) == msg + + +@pytest.mark.parametrize('version', [0, 1, 2, 3, 4]) +def test_parse_flexible_error(version): + # An unsupported request version returns v0 response. Make sure this works with all versions! + msg = ApiVersionsResponse[0](error_code=35, api_keys=[(18, 0, 3)]) + msg.with_header(correlation_id=1) + encoded = b'\x00\x00\x00\x10\x00\x00\x00\x01\x00#\x00\x00\x00\x01\x00\x12\x00\x00\x00\x03' + assert ApiVersionsResponse[version].decode(encoded, header=True, framed=True) == msg diff --git a/test/protocol/new/metadata/test_new_metadata.py b/test/protocol/new/metadata/test_new_metadata.py new file mode 100644 index 000000000..d26a7e6da --- /dev/null +++ b/test/protocol/new/metadata/test_new_metadata.py @@ -0,0 +1,63 @@ +import pytest + +from kafka.protocol.new.metadata import MetadataRequest, MetadataResponse + + +@pytest.mark.parametrize("version", range(MetadataRequest.min_version, MetadataRequest.max_version + 1)) +def test_metadata_request_roundtrip(version): + # MetadataRequest v0 is basic, v12 is flexible + topics = [ + MetadataRequest.MetadataRequestTopic(name="topic-1"), + MetadataRequest.MetadataRequestTopic(name="topic-2") + ] + # In v0-v3, allow_auto_topic_creation is not in schema, so it won't be encoded/decoded. + # But when we decode, the resulting data object will have the default value (True). + data = MetadataRequest( + topics=topics, + allow_auto_topic_creation=True, # Always True to match decoded default + include_cluster_authorized_operations=False, + include_topic_authorized_operations=True if version >= 8 else False + ) + + encoded = MetadataRequest.encode(data, version=version) + decoded = MetadataRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(MetadataResponse.min_version, MetadataResponse.max_version + 1)) +def test_metadata_response_roundtrip(version): + # Mock some data for response + brokers = [ + MetadataResponse.MetadataResponseBroker(node_id=1, host="localhost", port=9092, rack=None), + MetadataResponse.MetadataResponseBroker(node_id=2, host="localhost", port=9093, rack=None) + ] + topics = [ + MetadataResponse.MetadataResponseTopic( + error_code=0, + name="topic-1", + is_internal=False, + partitions=[ + MetadataResponse.MetadataResponseTopic.MetadataResponsePartition( + error_code=0, + partition_index=0, + leader_id=1, + replica_nodes=[1, 2], + isr_nodes=[1, 2], + offline_replicas=[] + ) + ], + authorized_operations=None + ) + ] + data = MetadataResponse( + throttle_time_ms=0, + brokers=brokers, + cluster_id=None, + controller_id=1 if version >= 1 else -1, + topics=topics, + authorized_operations=None + ) + + encoded = MetadataResponse.encode(data, version=version) + decoded = MetadataResponse.decode(encoded, version=version) + assert decoded == data diff --git a/test/protocol/new/producer/test_new_produce.py b/test/protocol/new/producer/test_new_produce.py new file mode 100644 index 000000000..0d8919e53 --- /dev/null +++ b/test/protocol/new/producer/test_new_produce.py @@ -0,0 +1,54 @@ +import pytest + +from kafka.protocol.new.producer import ProduceRequest, ProduceResponse + + +@pytest.mark.parametrize("version", range(ProduceRequest.min_version, ProduceRequest.max_version + 1)) +def test_produce_request_roundtrip(version): + topic_data = [ + ProduceRequest.TopicProduceData( + name="topic-1", + partition_data=[ + ProduceRequest.TopicProduceData.PartitionProduceData( + index=0, + records=b"some binary records data" + ) + ] + ) + ] + data = ProduceRequest( + transactional_id="trans-id" if version >= 3 else None, + acks=-1, + timeout_ms=500, + topic_data=topic_data + ) + + encoded = ProduceRequest.encode(data, version=version) + decoded = ProduceRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(ProduceResponse.min_version, ProduceResponse.max_version + 1)) +def test_produce_response_roundtrip(version): + responses = [ + ProduceResponse.TopicProduceResponse( + name="topic-1", + partition_responses=[ + ProduceResponse.TopicProduceResponse.PartitionProduceResponse( + index=0, + error_code=0, + base_offset=12345, + log_append_time_ms=1000 if version >= 2 else -1, + log_start_offset=100 if version >= 5 else -1 + ) + ] + ) + ] + data = ProduceResponse( + responses=responses, + throttle_time_ms=10 if version >= 1 else 0 + ) + + encoded = ProduceResponse.encode(data, version=version) + decoded = ProduceResponse.decode(encoded, version=version) + assert decoded == data diff --git a/test/protocol/new/producer/test_new_transactions.py b/test/protocol/new/producer/test_new_transactions.py new file mode 100644 index 000000000..49b332301 --- /dev/null +++ b/test/protocol/new/producer/test_new_transactions.py @@ -0,0 +1,228 @@ + +import pytest +from kafka.protocol.new.producer import ( + InitProducerIdRequest, InitProducerIdResponse, + AddPartitionsToTxnRequest, AddPartitionsToTxnResponse, + AddOffsetsToTxnRequest, AddOffsetsToTxnResponse, + EndTxnRequest, EndTxnResponse, + TxnOffsetCommitRequest, TxnOffsetCommitResponse, +) + +@pytest.mark.parametrize("version", range(InitProducerIdRequest.min_version, InitProducerIdRequest.max_version + 1)) +def test_init_producer_id_request_roundtrip(version): + data = InitProducerIdRequest( + transactional_id="test-txn", + transaction_timeout_ms=10000, + producer_id=-1, + producer_epoch=-1 + ) + encoded = InitProducerIdRequest.encode(data, version=version) + decoded = InitProducerIdRequest.decode(encoded, version=version) + assert decoded == data + +@pytest.mark.parametrize("version", range(InitProducerIdResponse.min_version, InitProducerIdResponse.max_version + 1)) +def test_init_producer_id_response_roundtrip(version): + data = InitProducerIdResponse( + throttle_time_ms=100 if version >= 1 else 0, + error_code=0, + producer_id=12345, + producer_epoch=1 + ) + encoded = InitProducerIdResponse.encode(data, version=version) + decoded = InitProducerIdResponse.decode(encoded, version=version) + assert decoded == data + +@pytest.mark.parametrize("version", range(AddPartitionsToTxnRequest.min_version, AddPartitionsToTxnRequest.max_version + 1)) +def test_add_partitions_to_txn_request_roundtrip(version): + if version < 4: + topics = [ + AddPartitionsToTxnRequest.AddPartitionsToTxnTopic( + name="topic-1", + partitions=[0, 1] + ) + ] + data = AddPartitionsToTxnRequest( + v3_and_below_transactional_id="test-txn", + v3_and_below_producer_id=12345, + v3_and_below_producer_epoch=1, + v3_and_below_topics=topics, + transactions=[] + ) + else: + # Use the specific data class for the Topics array within Transaction + Transaction = AddPartitionsToTxnRequest.AddPartitionsToTxnTransaction + Topic = Transaction.AddPartitionsToTxnTopic + transactions = [ + Transaction( + transactional_id="test-txn", + producer_id=12345, + producer_epoch=1, + verify_only=False, + topics=[ + Topic( + name="topic-1", + partitions=[0, 1] + ) + ] + ) + ] + data = AddPartitionsToTxnRequest( + transactions=transactions, + v3_and_below_transactional_id="", + v3_and_below_producer_id=0, + v3_and_below_producer_epoch=0, + v3_and_below_topics=[] + ) + encoded = AddPartitionsToTxnRequest.encode(data, version=version) + decoded = AddPartitionsToTxnRequest.decode(encoded, version=version) + assert decoded == data + +@pytest.mark.parametrize("version", range(AddPartitionsToTxnResponse.min_version, AddPartitionsToTxnResponse.max_version + 1)) +def test_add_partitions_to_txn_response_roundtrip(version): + if version < 4: + TopicResult = AddPartitionsToTxnResponse.AddPartitionsToTxnTopicResult + PartitionResult = TopicResult.AddPartitionsToTxnPartitionResult + topic_results = [ + TopicResult( + name="topic-1", + results_by_partition=[ + PartitionResult( + partition_index=0, + partition_error_code=0 + ) + ] + ) + ] + data = AddPartitionsToTxnResponse( + throttle_time_ms=100 if version >= 1 else 0, + results_by_topic_v3_and_below=topic_results, + results_by_transaction=[] + ) + else: + Result = AddPartitionsToTxnResponse.AddPartitionsToTxnResult + TopicResult = Result.AddPartitionsToTxnTopicResult + PartitionResult = TopicResult.AddPartitionsToTxnPartitionResult + topic_results = [ + TopicResult( + name="topic-1", + results_by_partition=[ + PartitionResult( + partition_index=0, + partition_error_code=0 + ) + ] + ) + ] + results_by_transaction = [ + Result( + transactional_id="test-txn", + topic_results=topic_results + ) + ] + data = AddPartitionsToTxnResponse( + throttle_time_ms=100, + error_code=0, + results_by_transaction=results_by_transaction, + results_by_topic_v3_and_below=[] + ) + encoded = AddPartitionsToTxnResponse.encode(data, version=version) + decoded = AddPartitionsToTxnResponse.decode(encoded, version=version) + assert decoded == data + +@pytest.mark.parametrize("version", range(AddOffsetsToTxnRequest.min_version, AddOffsetsToTxnRequest.max_version + 1)) +def test_add_offsets_to_txn_request_roundtrip(version): + data = AddOffsetsToTxnRequest( + transactional_id="test-txn", + producer_id=12345, + producer_epoch=1, + group_id="test-group" + ) + encoded = AddOffsetsToTxnRequest.encode(data, version=version) + decoded = AddOffsetsToTxnRequest.decode(encoded, version=version) + assert decoded == data + +@pytest.mark.parametrize("version", range(AddOffsetsToTxnResponse.min_version, AddOffsetsToTxnResponse.max_version + 1)) +def test_add_offsets_to_txn_response_roundtrip(version): + data = AddOffsetsToTxnResponse( + throttle_time_ms=100 if version >= 1 else 0, + error_code=0 + ) + encoded = AddOffsetsToTxnResponse.encode(data, version=version) + decoded = AddOffsetsToTxnResponse.decode(encoded, version=version) + assert decoded == data + +@pytest.mark.parametrize("version", range(EndTxnRequest.min_version, EndTxnRequest.max_version + 1)) +def test_end_txn_request_roundtrip(version): + data = EndTxnRequest( + transactional_id="test-txn", + producer_id=12345, + producer_epoch=1, + committed=True + ) + encoded = EndTxnRequest.encode(data, version=version) + decoded = EndTxnRequest.decode(encoded, version=version) + assert decoded == data + +@pytest.mark.parametrize("version", range(EndTxnResponse.min_version, EndTxnResponse.max_version + 1)) +def test_end_txn_response_roundtrip(version): + data = EndTxnResponse( + throttle_time_ms=100 if version >= 1 else 0, + error_code=0 + ) + encoded = EndTxnResponse.encode(data, version=version) + decoded = EndTxnResponse.decode(encoded, version=version) + assert decoded == data + +@pytest.mark.parametrize("version", range(TxnOffsetCommitRequest.min_version, TxnOffsetCommitRequest.max_version + 1)) +def test_txn_offset_commit_request_roundtrip(version): + Topic = TxnOffsetCommitRequest.TxnOffsetCommitRequestTopic + Partition = Topic.TxnOffsetCommitRequestPartition + topics = [ + Topic( + name="topic-1", + partitions=[ + Partition( + partition_index=0, + committed_offset=100, + committed_leader_epoch=1 if version >= 2 else -1, + committed_metadata="meta" + ) + ] + ) + ] + data = TxnOffsetCommitRequest( + transactional_id="test-txn", + group_id="test-group", + producer_id=12345, + producer_epoch=1, + generation_id=1 if version >= 3 else -1, + member_id="test-member" if version >= 3 else "", + group_instance_id=None, + topics=topics + ) + encoded = TxnOffsetCommitRequest.encode(data, version=version) + decoded = TxnOffsetCommitRequest.decode(encoded, version=version) + assert decoded == data + +@pytest.mark.parametrize("version", range(TxnOffsetCommitResponse.min_version, TxnOffsetCommitResponse.max_version + 1)) +def test_txn_offset_commit_response_roundtrip(version): + Topic = TxnOffsetCommitResponse.TxnOffsetCommitResponseTopic + Partition = Topic.TxnOffsetCommitResponsePartition + topics = [ + Topic( + name="topic-1", + partitions=[ + Partition( + partition_index=0, + error_code=0 + ) + ] + ) + ] + data = TxnOffsetCommitResponse( + throttle_time_ms=100 if version >= 1 else 0, + topics=topics + ) + encoded = TxnOffsetCommitResponse.encode(data, version=version) + decoded = TxnOffsetCommitResponse.decode(encoded, version=version) + assert decoded == data diff --git a/test/protocol/new/schemas/test_field.py b/test/protocol/new/schemas/test_field.py new file mode 100644 index 000000000..2a069ea99 --- /dev/null +++ b/test/protocol/new/schemas/test_field.py @@ -0,0 +1,116 @@ +import pytest + +from kafka.protocol.new.schemas.fields.array import ArrayField +from kafka.protocol.new.schemas.fields.struct import StructField +from kafka.protocol.new.schemas.fields.base import BaseField +from kafka.protocol.new.schemas.fields.simple import SimpleField +from kafka.protocol.types import Int16, Int32, Boolean, String, UUID + + +def test_parse_versions(): + assert BaseField.parse_versions("0+") == (0, 32767) + assert BaseField.parse_versions("1-3") == (1, 3) + assert BaseField.parse_versions("5") == (5, 5) + assert BaseField.parse_versions("none") == (-1, -1) + assert BaseField.parse_versions(None) is None + assert BaseField.parse_versions("") is None + + +def test_field_int_defaults(): + # Test literal int default + f1 = BaseField.parse_json({"name": "f1", "versions": "0+", "type": "int32", "default": -1}) + assert f1.default == -1 + + # Test string int default + f2 = BaseField.parse_json({"name": "f2", "versions": "0+", "type": "int32", "default": "-1"}) + assert f2.default == -1 + + # Test hex string default + f3 = BaseField.parse_json({"name": "f3", "versions": "0+", "type": "int32", "default": "0x7fffffff"}) + assert f3.default == 2147483647 + + # Test empty string default (should be 0 for int) + f4 = BaseField.parse_json({"name": "f4", "versions": "0+", "type": "int32", "default": ""}) + assert f4.default == 0 + + +def test_field_bool_defaults(): + # Test literal bool default + f1 = BaseField.parse_json({"name": "f1", "versions": "0+", "type": "bool", "default": True}) + assert f1.default is True + + # Test string bool default + f2 = BaseField.parse_json({"name": "f2", "versions": "0+", "type": "bool", "default": "true"}) + assert f2.default is True + f3 = BaseField.parse_json({"name": "f3", "versions": "0+", "type": "bool", "default": "False"}) + assert f3.default is False + + # Test empty string default (should be False for bool) + f4 = BaseField.parse_json({"name": "f4", "versions": "0+", "type": "bool", "default": ""}) + assert f4.default is False + + +def test_field_string_defaults(): + f1 = BaseField.parse_json({"name": "f1", "versions": "0+", "type": "string", "default": "foo"}) + assert f1.default == "foo" + + f2 = BaseField.parse_json({"name": "f2", "versions": "0+", "type": "string", "default": ""}) + assert f2.default == "" + + # Nullable string + f3 = BaseField.parse_json({ + "name": "f3", + "type": "string", + "default": "null", + "nullableVersions": "0+", + "versions": "0+" + }) + assert f3.default is None + + +@pytest.mark.parametrize(('field_json', 'inner_type'), [ + ({"name": "f", "versions": "0+", "type": "int16"}, Int16), + ({"name": "f", "versions": "0+", "type": "int32"}, Int32), + ({"name": "f", "versions": "0+", "type": "string"}, String), + ({"name": "f", "versions": "0+", "type": "bool"}, Boolean), + ({"name": "f", "versions": "0+", "type": "uuid"}, UUID), +]) +def test_field_basic_types(field_json, inner_type): + field = BaseField.parse_json(field_json) + assert isinstance(field, SimpleField) + assert field._type is inner_type or isinstance(field._type, inner_type) + + +def test_struct_field(): + field = BaseField.parse_json({"name": "f", "versions": "0+", "type": "Foo", "fields": [{"name": "b", "versions": "0+", "type": "int16"}]}) + assert field.is_struct() + assert isinstance(field, StructField) + assert isinstance(field.fields['b'], SimpleField) + assert field.fields['b']._type is Int16 + + +def test_array_field(): + field = BaseField.parse_json({"name": "f", "versions": "0+", "type": "[]int16"}) + assert field.is_array() + assert not field.is_struct_array() + assert isinstance(field, ArrayField) + assert isinstance(field.array_of, SimpleField) + assert field.array_of._type is Int16 + + +def test_struct_array_field(): + field = BaseField.parse_json({"name": "f", "versions": "0+", "type": "[]Foo", "fields": [{"name": "b", "versions": "0+", "type": "int16"}]}) + assert field.is_array() + assert field.is_struct_array() + assert isinstance(field, ArrayField) + assert isinstance(field.array_of, StructField) + assert field.array_of.fields['b']._type is Int16 + + +def test_field_version_check(): + f = BaseField.parse_json({"name": "f", "type": "int32", "versions": "1-3"}) + assert not f.for_version_q(0) + assert f.for_version_q(1) + assert f.for_version_q(2) + assert f.for_version_q(3) + assert not f.for_version_q(4) diff --git a/test/protocol/new/schemas/test_load_json.py b/test/protocol/new/schemas/test_load_json.py new file mode 100644 index 000000000..b1b82a3e1 --- /dev/null +++ b/test/protocol/new/schemas/test_load_json.py @@ -0,0 +1,29 @@ +from kafka.protocol.new.schemas import load_json + + +def test_load_json_common_structs(): + # AddPartitionsToTxnRequest uses commonStructs for 'AddPartitionsToTxnTopic' + data = load_json('AddPartitionsToTxnRequest') + + assert 'commonStructs' in data + common_names = [s['name'] for s in data['commonStructs']] + assert 'AddPartitionsToTxnTopic' in common_names + + # Find V3AndBelowTopics field - it should have fields injected + v3_topics = next(f for f in data['fields'] if f['name'] == 'V3AndBelowTopics') + assert 'fields' in v3_topics + assert any(f['name'] == 'Partitions' for f in v3_topics['fields']) + + # Find Transactions field (v4+) + transactions = next(f for f in data['fields'] if f['name'] == 'Transactions') + # It has a nested 'Topics' field which should also have fields injected + nested_topics = next(f for f in transactions['fields'] if f['name'] == 'Topics') + assert 'fields' in nested_topics + assert any(f['name'] == 'Partitions' for f in nested_topics['fields']) + + +def test_load_json_basic(): + # Simple check for a message without commonStructs + data = load_json('ApiVersionsRequest') + assert 'fields' in data + assert 'commonStructs' not in data or len(data['commonStructs']) == 0 diff --git a/test/protocol/new/test_api_compatibility.py b/test/protocol/new/test_api_compatibility.py new file mode 100644 index 000000000..41ebea33e --- /dev/null +++ b/test/protocol/new/test_api_compatibility.py @@ -0,0 +1,308 @@ +import pytest +import os +import json +from io import BytesIO + +from kafka.protocol.api import RequestHeaderV2 +from kafka.protocol.api_versions import ( + ApiVersionsRequest_v3, ApiVersionsResponse_v3, + ApiVersionsRequest_v4, ApiVersionsResponse_v4 +) +from kafka.protocol.new.metadata import ( + ApiVersionsRequest as NewApiVersionsRequest, + ApiVersionsResponse as NewApiVersionsResponse +) + +# --- Golden Samples (Generated from existing working system) --- + +# ApiVersionsRequest_v3 (with a client_id for the header) +# client_software_name = "kafka-python" +# client_software_version = "2.9.0" +# tags = {} +# Full request: RequestHeaderV2 + Request body +# RequestHeaderV2: api_key=18, api_version=3, correlation_id=1, client_id='test-client', tags={} +# api_key (Int16) = 18 (0x0012) +# api_version (Int16) = 3 (0x0003) +# correlation_id (Int32) = 1 (0x00000001) +# client_id (String) = len(11) + b'test-client' -> 0x000B + b'test-client' +# tags (TaggedFields) = 0x00 (empty) +# Request Body: +# client_software_name (CompactString) = len(12) + b'kafka-python' -> 0x0D + b'kafka-python' +# client_software_version (CompactString) = len(5) + b'2.9.0' -> 0x06 + b'2.9.0' +# tags (TaggedFields) = 0x00 (empty) +GOLDEN_API_VERSIONS_REQUEST_V3_BYTES = \ + b'\x00\x12' \ + b'\x00\x03' \ + b'\x00\x00\x00\x01' \ + b'\x00\x0btest-client' \ + b'\x00' \ + b'\x0dkafka-python' \ + b'\x062.9.0' \ + b'\x00' + +# ApiVersionsResponse_v3 (with no header for simplicity in this golden sample) +# error_code = 0 +# api_versions = [ +# {'api_key': 0, 'min_version': 0, 'max_version': 9, 'tags': {}}, # Produce +# {'api_key': 1, 'min_version': 0, 'max_version': 10, 'tags': {}}, # Fetch +# ] +# throttle_time_ms = 0 +# tags = {} +# Response Body: +# error_code (Int16) = 0 (0x0000) +# api_versions (CompactArray): +# len(2) -> 0x03 (VarInt) +# Item 1: +# api_key (Int16) = 0 (0x0000) +# min_version (Int16) = 0 (0x0000) +# max_version (Int16) = 9 (0x0009) +# tags (TaggedFields) = 0x00 (empty) +# Item 2: +# api_key (Int16) = 1 (0x0001) +# min_version (Int16) = 0 (0x0000) +# max_version (Int16) = 10 (0x000A) +# tags (TaggedFields) = 0x00 (empty) +# throttle_time_ms (Int32) = 0 (0x00000000) +# tags (TaggedFields) = 0x00 (empty) +GOLDEN_API_VERSIONS_RESPONSE_V3_BODY_BYTES = \ + b'\x00\x00' \ + b'\x03' \ + b'\x00\x00\x00\x00\x00\x09\x00' \ + b'\x00\x01\x00\x00\x00\x0a\x00' \ + b'\x00\x00\x00\x00' \ + b'\x00' + + +class TestApiCompatibility: + + def test_old_system_encode_decode_request_v3(self): + # Old system encoding + request = ApiVersionsRequest_v3( + client_software_name="kafka-python", + client_software_version="2.9.0", + tags={} # This needs to be passed explicitly for older Structs + ) + # Manually build header for old system to match golden bytes + header = RequestHeaderV2( + api_key=request.API_KEY, + api_version=request.API_VERSION, + correlation_id=1, + client_id='test-client', + tags={} + ) + encoded_header = header.encode() + encoded_body = request.encode() + full_encoded_request = encoded_header + encoded_body + + assert full_encoded_request == GOLDEN_API_VERSIONS_REQUEST_V3_BYTES + + # Old system decoding + decoded_header = RequestHeaderV2.decode(BytesIO(full_encoded_request)) + decoded_request = ApiVersionsRequest_v3.decode( + BytesIO(full_encoded_request[len(encoded_header):]) + ) + + assert decoded_header.api_key == request.API_KEY + assert decoded_header.api_version == request.API_VERSION + assert decoded_header.correlation_id == 1 + assert decoded_header.client_id == 'test-client' + assert decoded_header.tags == {} + + assert decoded_request.client_software_name == "kafka-python" + assert decoded_request.client_software_version == "2.9.0" + assert decoded_request.tags == {} # should be empty dict by default from Struct + + def test_new_system_encode_decode_request_v3(self): + # New system encoding + request_data = NewApiVersionsRequest( + client_software_name="kafka-python", + client_software_version="2.9.0" + ) + # New system does not manage the header directly in encode/decode calls + # We need to construct the full message with a header for comparison + # against the golden sample which includes a header. + # Header is managed by the client_async / conn layer which uses + # the _tagged_fields / tags distinction from api.py + # For now, let's just test the body encoding/decoding. + # This simplifies the comparison. + + # The new system's encode method for ApiMessage directly encodes the message body + # without the request header. + new_system_encoded_body = NewApiVersionsRequest.encode(request_data, version=3) + + # To compare against GOLDEN_API_VERSIONS_REQUEST_V3_BYTES, which includes a header, + # we need to extract the body part from the golden bytes. + # Length of header: Int16 + Int16 + Int32 + CompactString('test-client') + TaggedFields({}) + # 2 + 2 + 4 + (1+11) + 1 = 21 bytes + # This is brittle. Better to generate golden body bytes directly. + + # Let's generate golden body bytes directly from the old system. + old_request_body_instance = ApiVersionsRequest_v3( + client_software_name="kafka-python", + client_software_version="2.9.0", + tags={} + ) + old_system_golden_body_bytes = old_request_body_instance.encode() + assert new_system_encoded_body == old_system_golden_body_bytes + + # New system decoding + new_system_decoded_data = NewApiVersionsRequest.decode( + BytesIO(old_system_golden_body_bytes), version=3 + ) + assert new_system_decoded_data.client_software_name == "kafka-python" + assert new_system_decoded_data.client_software_version == "2.9.0" + assert new_system_decoded_data.tags is None + + def test_old_system_encode_decode_response_v3(self): + # Old system encoding + response = ApiVersionsResponse_v3( + error_code=0, + api_versions=[ + #{'api_key': 0, 'min_version': 0, 'max_version': 9, 'tags': {}}, + (0, 0, 9, {}), + #{'api_key': 1, 'min_version': 0, 'max_version': 10, 'tags': {}} + (1, 0, 10, {}) + ], + throttle_time_ms=0, + tags={} + ) + encoded_body = response.encode() + assert encoded_body == GOLDEN_API_VERSIONS_RESPONSE_V3_BODY_BYTES + + # Old system decoding + decoded_response = ApiVersionsResponse_v3.decode(BytesIO(encoded_body)) + + assert decoded_response.error_code == 0 + assert len(decoded_response.api_versions) == 2 + assert decoded_response.api_versions[0][0] == 0 # api_key + assert decoded_response.api_versions[0][1] == 0 # min_version + assert decoded_response.api_versions[0][2] == 9 # max_version + #assert decoded_response.api_versions[0].tags == {} + assert decoded_response.api_versions[1][0] == 1 # api_key + assert decoded_response.api_versions[1][1] == 0 # min_version + assert decoded_response.api_versions[1][2] == 10 # max_version + #assert decoded_response.api_versions[1].tags == {} + assert decoded_response.throttle_time_ms == 0 + #assert decoded_response.tags == {} + + def test_new_system_encode_decode_response_v3(self): + # New system encoding + response_data = NewApiVersionsResponse( + error_code=0, + api_keys=[ + NewApiVersionsResponse.fields['api_keys']( + api_key=0, min_version=0, max_version=9 + ), + NewApiVersionsResponse.fields['api_keys']( + api_key=1, min_version=0, max_version=10 + ) + ], + throttle_time_ms=0 + ) + + # Verify nested field access + assert 'min_version' in NewApiVersionsResponse.fields['api_keys'].fields + min_ver_field = NewApiVersionsResponse.fields['api_keys'].fields['min_version'] + assert min_ver_field.name == 'min_version' + # assert min_ver_field.type == Int16 + + new_system_encoded_body = NewApiVersionsResponse.encode(response_data, version=3) + assert new_system_encoded_body == GOLDEN_API_VERSIONS_RESPONSE_V3_BODY_BYTES + + # New system decoding + new_system_decoded_data = NewApiVersionsResponse.decode( + BytesIO(GOLDEN_API_VERSIONS_RESPONSE_V3_BODY_BYTES), version=3 + ) + assert new_system_decoded_data.error_code == 0 + assert len(new_system_decoded_data.api_keys) == 2 + assert new_system_decoded_data.api_keys[0].api_key == 0 + assert new_system_decoded_data.api_keys[0].min_version == 0 + assert new_system_decoded_data.api_keys[0].max_version == 9 + assert new_system_decoded_data.api_keys[0].tags is None + assert new_system_decoded_data.api_keys[1].api_key == 1 + assert new_system_decoded_data.api_keys[1].min_version == 0 + assert new_system_decoded_data.api_keys[1].max_version == 10 + assert new_system_decoded_data.api_keys[1].tags is None + assert new_system_decoded_data.throttle_time_ms == 0 + assert new_system_decoded_data.tags is None + + def test_tagged_fields_retention(self): + # 1. Verify no tags present (None) + # Using GOLDEN_API_VERSIONS_RESPONSE_V3_BODY_BYTES which has empty tags + decoded_data = NewApiVersionsResponse.decode( + BytesIO(GOLDEN_API_VERSIONS_RESPONSE_V3_BODY_BYTES), version=3 + ) + assert decoded_data.tags is None + assert decoded_data.unknown_tags is None + + # 2. Verify explicit tags present + # ApiVersionsResponse_v3 has 'throttle_time_ms' (not tagged) and 'supported_features' (tagged, tag=0) + # Let's manually construct a response with supported_features set + # Using schema definition: SupportedFeatures (tag=0) -> []SupportedFeatureKey + # SupportedFeatureKey: Name (compact string), MinVersion (int16), MaxVersion (int16) + + # We need to construct bytes for this. + # Base: 0 error code, 0 api_keys (empty array), 0 throttle_time_ms + # Tags: 1 tag (tag 0) -> SupportedFeatures array + + # Body: + # error_code (Int16): 0 -> \x00\x00 + # api_keys (CompactArray): 0 length -> \x01 (varint 0+1? No, varint len+1. So 0 len is \x01) + # throttle_time_ms (Int32): 0 -> \x00\x00\x00\x00 + # tags (TaggedFields): + # num_fields (varint): 1 + # tag (varint): 0 (SupportedFeatures) + # size (varint): length of data + # data: CompactArray of SupportedFeatureKey + + # SupportedFeatureKey data: + # CompactArray len 1 -> \x02 + # Item 1: + # Name: "feat" -> \x05feat (len 4+1) + # MinVersion: 1 -> \x00\x01 + # MaxVersion: 2 -> \x00\x02 + # Tags: empty -> \x00 + # Total item len: 1+4 + 2 + 2 + 1 = 10 bytes + # Array data len: 1 (len byte) + 10 = 11 bytes + + # Tag size: 11 + + tagged_bytes = ( + b'\x00\x00' # error_code + b'\x01' # api_keys (empty) + b'\x00\x00\x00\x00' # throttle_time_ms + b'\x01' # num_tags = 1 + b'\x00' # tag = 0 (SupportedFeatures) + b'\x0b' # size = 11 + b'\x02' # array len = 1 + b'\x05feat' # Name + b'\x00\x01' # MinVersion + b'\x00\x02' # MaxVersion + b'\x00' # Item tags + ) + + decoded_tagged = NewApiVersionsResponse.decode(BytesIO(tagged_bytes), version=3) + assert decoded_tagged.tags == {'supported_features'} + assert decoded_tagged.supported_features is not None + assert len(decoded_tagged.supported_features) == 1 + assert decoded_tagged.supported_features[0].name == 'feat' + assert decoded_tagged.unknown_tags is None + + # 3. Verify unknown tags + # Same base, but tag = 99 (unknown) + # data: \x01\x02\x03 (3 bytes) + unknown_tagged_bytes = ( + b'\x00\x00' # error_code + b'\x01' # api_keys (empty) + b'\x00\x00\x00\x00' # throttle_time_ms + b'\x01' # num_tags = 1 + b'\x63' # tag = 99 (varint \x63) + b'\x03' # size = 3 + b'\x01\x02\x03' # data + ) + + decoded_unknown = NewApiVersionsResponse.decode(BytesIO(unknown_tagged_bytes), version=3) + assert decoded_unknown.tags is None # No known tags explicitly set + assert decoded_unknown.unknown_tags is not None + assert '_99' in decoded_unknown.unknown_tags + assert decoded_unknown.unknown_tags['_99'] == b'\x01\x02\x03' diff --git a/test/protocol/new/test_new_parser.py b/test/protocol/new/test_new_parser.py new file mode 100644 index 000000000..f53a96177 --- /dev/null +++ b/test/protocol/new/test_new_parser.py @@ -0,0 +1,243 @@ +import pytest + +from kafka.protocol.parser import KafkaProtocol + +from kafka.protocol.new.metadata import ( + ApiVersionsRequest, ApiVersionsResponse, + FindCoordinatorRequest, FindCoordinatorResponse, + MetadataRequest, MetadataResponse, +) +from kafka.protocol.new.consumer import ( + JoinGroupRequest, JoinGroupResponse, + SyncGroupRequest, SyncGroupResponse, + LeaveGroupRequest, LeaveGroupResponse, + HeartbeatRequest, HeartbeatResponse, + OffsetCommitRequest, OffsetCommitResponse, + OffsetFetchRequest, OffsetFetchResponse +) + + +TEST_API_VERSIONS_1 = { + 'client_id': '_internal_client_kYVL', + 'messages': ( + { + 'request': ( + ApiVersionsRequest[4](client_software_name='kafka-python', client_software_version='3.0.0'), + b'\x00\x00\x004\x00\x12\x00\x04\x00\x00\x00\x01\x00\x15_internal_client_kYVL\x00\rkafka-python\x063.0.0\x00', + ), + 'response': ( + b'\x00\x00\x00\x10\x00\x00\x00\x01\x00#\x00\x00\x00\x01\x00\x12\x00\x00\x00\x03', + 1, + ApiVersionsResponse[0](error_code=35, api_keys=[(18, 0, 3)]) # Note different class on error - special case + ), + }, + { + 'request': ( + ApiVersionsRequest[3](client_software_name='kafka-python', client_software_version='3.0.0'), + b'\x00\x00\x004\x00\x12\x00\x03\x00\x00\x00\x02\x00\x15_internal_client_kYVL\x00\rkafka-python\x063.0.0\x00', + ), + 'response': ( + b'\x00\x00\x01j\x00\x00\x00\x02\x00\x003\x00\x00\x00\x00\x00\x08\x00\x00\x01\x00\x00\x00\x0b\x00\x00\x02\x00\x00\x00\x05\x00\x00\x03\x00\x00\x00\t\x00\x00\x04\x00\x00\x00\x04\x00\x00\x05\x00\x00\x00\x03\x00\x00\x06\x00\x00\x00\x06\x00\x00\x07\x00\x00\x00\x03\x00\x00\x08\x00\x00\x00\x08\x00\x00\t\x00\x00\x00\x07\x00\x00\n\x00\x00\x00\x03\x00\x00\x0b\x00\x00\x00\x07\x00\x00\x0c\x00\x00\x00\x04\x00\x00\r\x00\x00\x00\x04\x00\x00\x0e\x00\x00\x00\x05\x00\x00\x0f\x00\x00\x00\x05\x00\x00\x10\x00\x00\x00\x04\x00\x00\x11\x00\x00\x00\x01\x00\x00\x12\x00\x00\x00\x03\x00\x00\x13\x00\x00\x00\x05\x00\x00\x14\x00\x00\x00\x04\x00\x00\x15\x00\x00\x00\x02\x00\x00\x16\x00\x00\x00\x03\x00\x00\x17\x00\x00\x00\x03\x00\x00\x18\x00\x00\x00\x01\x00\x00\x19\x00\x00\x00\x01\x00\x00\x1a\x00\x00\x00\x01\x00\x00\x1b\x00\x00\x00\x00\x00\x00\x1c\x00\x00\x00\x03\x00\x00\x1d\x00\x00\x00\x02\x00\x00\x1e\x00\x00\x00\x02\x00\x00\x1f\x00\x00\x00\x02\x00\x00 \x00\x00\x00\x03\x00\x00!\x00\x00\x00\x01\x00\x00"\x00\x00\x00\x01\x00\x00#\x00\x00\x00\x02\x00\x00$\x00\x00\x00\x02\x00\x00%\x00\x00\x00\x02\x00\x00&\x00\x00\x00\x02\x00\x00\'\x00\x00\x00\x02\x00\x00(\x00\x00\x00\x02\x00\x00)\x00\x00\x00\x02\x00\x00*\x00\x00\x00\x02\x00\x00+\x00\x00\x00\x02\x00\x00,\x00\x00\x00\x01\x00\x00-\x00\x00\x00\x00\x00\x00.\x00\x00\x00\x00\x00\x00/\x00\x00\x00\x00\x00\x000\x00\x00\x00\x00\x00\x001\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00', + 2, + ApiVersionsResponse[3](error_code=0, api_keys=[(0, 0, 8), (1, 0, 11), (2, 0, 5), (3, 0, 9), (4, 0, 4), (5, 0, 3), (6, 0, 6), (7, 0, 3), (8, 0, 8), (9, 0, 7), (10, 0, 3), (11, 0, 7), (12, 0, 4), (13, 0, 4), (14, 0, 5), (15, 0, 5), (16, 0, 4), (17, 0, 1), (18, 0, 3), (19, 0, 5), (20, 0, 4), (21, 0, 2), (22, 0, 3), (23, 0, 3), (24, 0, 1), (25, 0, 1), (26, 0, 1), (27, 0, 0), (28, 0, 3), (29, 0, 2), (30, 0, 2), (31, 0, 2), (32, 0, 3), (33, 0, 1), (34, 0, 1), (35, 0, 2), (36, 0, 2), (37, 0, 2), (38, 0, 2), (39, 0, 2), (40, 0, 2), (41, 0, 2), (42, 0, 2), (43, 0, 2), (44, 0, 1), (45, 0, 0), (46, 0, 0), (47, 0, 0), (48, 0, 0), (49, 0, 0)], throttle_time_ms=0), + ), + }, + ), +} + + +TEST_API_VERSIONS_2 = { + 'client_id': 'admin_client_VIsf', + 'messages': ( + { + 'request': ( + ApiVersionsRequest[4](client_software_name='kafka-python', client_software_version='3.0.0'), + b'\x00\x00\x000\x00\x12\x00\x04\x00\x00\x00\x01\x00\x11admin_client_VIsf\x00\rkafka-python\x063.0.0\x00', + ), + 'response': ( + b'\x00\x00\x00\x10\x00\x00\x00\x01\x00#\x00\x00\x00\x01\x00\x12\x00\x00\x00\x03', + 1, + ApiVersionsResponse[0](error_code=35, api_keys=[(18, 0, 3)]) # Note different class on error - special case + ), + }, + { + 'request': ( + ApiVersionsRequest[3](client_software_name='kafka-python', client_software_version='3.0.0'), + b'\x00\x00\x000\x00\x12\x00\x03\x00\x00\x00\x02\x00\x11admin_client_VIsf\x00\rkafka-python\x063.0.0\x00', + ), + 'response': ( + b'\x00\x00\x01j\x00\x00\x00\x02\x00\x003\x00\x00\x00\x00\x00\x08\x00\x00\x01\x00\x00\x00\x0b\x00\x00\x02\x00\x00\x00\x05\x00\x00\x03\x00\x00\x00\t\x00\x00\x04\x00\x00\x00\x04\x00\x00\x05\x00\x00\x00\x03\x00\x00\x06\x00\x00\x00\x06\x00\x00\x07\x00\x00\x00\x03\x00\x00\x08\x00\x00\x00\x08\x00\x00\t\x00\x00\x00\x07\x00\x00\n\x00\x00\x00\x03\x00\x00\x0b\x00\x00\x00\x07\x00\x00\x0c\x00\x00\x00\x04\x00\x00\r\x00\x00\x00\x04\x00\x00\x0e\x00\x00\x00\x05\x00\x00\x0f\x00\x00\x00\x05\x00\x00\x10\x00\x00\x00\x04\x00\x00\x11\x00\x00\x00\x01\x00\x00\x12\x00\x00\x00\x03\x00\x00\x13\x00\x00\x00\x05\x00\x00\x14\x00\x00\x00\x04\x00\x00\x15\x00\x00\x00\x02\x00\x00\x16\x00\x00\x00\x03\x00\x00\x17\x00\x00\x00\x03\x00\x00\x18\x00\x00\x00\x01\x00\x00\x19\x00\x00\x00\x01\x00\x00\x1a\x00\x00\x00\x01\x00\x00\x1b\x00\x00\x00\x00\x00\x00\x1c\x00\x00\x00\x03\x00\x00\x1d\x00\x00\x00\x02\x00\x00\x1e\x00\x00\x00\x02\x00\x00\x1f\x00\x00\x00\x02\x00\x00 \x00\x00\x00\x03\x00\x00!\x00\x00\x00\x01\x00\x00"\x00\x00\x00\x01\x00\x00#\x00\x00\x00\x02\x00\x00$\x00\x00\x00\x02\x00\x00%\x00\x00\x00\x02\x00\x00&\x00\x00\x00\x02\x00\x00\'\x00\x00\x00\x02\x00\x00(\x00\x00\x00\x02\x00\x00)\x00\x00\x00\x02\x00\x00*\x00\x00\x00\x02\x00\x00+\x00\x00\x00\x02\x00\x00,\x00\x00\x00\x01\x00\x00-\x00\x00\x00\x00\x00\x00.\x00\x00\x00\x00\x00\x00/\x00\x00\x00\x00\x00\x000\x00\x00\x00\x00\x00\x001\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00', + 2, + ApiVersionsResponse[3](error_code=0, api_keys=[(0, 0, 8), (1, 0, 11), (2, 0, 5), (3, 0, 9), (4, 0, 4), (5, 0, 3), (6, 0, 6), (7, 0, 3), (8, 0, 8), (9, 0, 7), (10, 0, 3), (11, 0, 7), (12, 0, 4), (13, 0, 4), (14, 0, 5), (15, 0, 5), (16, 0, 4), (17, 0, 1), (18, 0, 3), (19, 0, 5), (20, 0, 4), (21, 0, 2), (22, 0, 3), (23, 0, 3), (24, 0, 1), (25, 0, 1), (26, 0, 1), (27, 0, 0), (28, 0, 3), (29, 0, 2), (30, 0, 2), (31, 0, 2), (32, 0, 3), (33, 0, 1), (34, 0, 1), (35, 0, 2), (36, 0, 2), (37, 0, 2), (38, 0, 2), (39, 0, 2), (40, 0, 2), (41, 0, 2), (42, 0, 2), (43, 0, 2), (44, 0, 1), (45, 0, 0), (46, 0, 0), (47, 0, 0), (48, 0, 0), (49, 0, 0)], throttle_time_ms=0), + ), + }, + { + 'request': ( + MetadataRequest[8](topics=[], allow_auto_topic_creation=True), + b'\x00\x00\x00"\x00\x03\x00\x08\x00\x00\x00\x03\x00\x11admin_client_VIsf\x00\x00\x00\x00\x01\x00\x00', + ), + 'response': ( + b'\x00\x00\x00E\x00\x00\x00\x03\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x00\tlocalhost\x00\x00\xb9}\xff\xff\x00\x1634wjNp3hRJixCRvMlK9Znw\x00\x00\x00\x00\x00\x00\x00\x00\x80\x00\x00\x00', + 3, + MetadataResponse[8](throttle_time_ms=0, brokers=[(0, 'localhost', 47485, None)], cluster_id='34wjNp3hRJixCRvMlK9Znw', controller_id=0, topics=[], authorized_operations=None), + ), + }, + { + 'request': ( + MetadataRequest[8](topics=None, allow_auto_topic_creation=True), + b'\x00\x00\x00"\x00\x03\x00\x08\x00\x00\x00\x04\x00\x11admin_client_VIsf\xff\xff\xff\xff\x01\x00\x00', + ), + 'response': ( + b'\x00\x00\x00E\x00\x00\x00\x04\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x00\tlocalhost\x00\x00\xb9}\xff\xff\x00\x1634wjNp3hRJixCRvMlK9Znw\x00\x00\x00\x00\x00\x00\x00\x00\x80\x00\x00\x00', + 4, + MetadataResponse[8](throttle_time_ms=0, brokers=[(0, 'localhost', 47485, None)], cluster_id='34wjNp3hRJixCRvMlK9Znw', controller_id=0, topics=[], authorized_operations=None), + ), + } + ), +} + + +TEST_FIND_COORDINATOR = { + 'client_id': 'consumer_thread-1', + 'messages': ( + { + 'request': ( + FindCoordinatorRequest[2](key='test-group-CYErjI'), + b'\x00\x00\x00/\x00\n\x00\x02\x00\x00\x00\x01\x00\x11consumer_thread-1\x00\x11test-group-CYErjI\x00', + ), + 'response': ( + b'\x00\x00\x007\x00\x00\x00\x01\x00\x00\x00\x00\x00\x0f\x00!The coordinator is not available.\xff\xff\xff\xff\x00\x00\xff\xff\xff\xff', + 1, + FindCoordinatorResponse[2](throttle_time_ms=0, error_code=15, error_message='The coordinator is not available.', node_id=-1, host='', port=-1), + ), + }, + { + 'request': ( + FindCoordinatorRequest[2](key='test-group-CYErjI'), + b'\x00\x00\x00/\x00\n\x00\x02\x00\x00\x00\x02\x00\x11consumer_thread-1\x00\x11test-group-CYErjI\x00', + ), + 'response': ( + b'\x00\x00\x00#\x00\x00\x00\x02\x00\x00\x00\x00\x00\x00\x00\x04NONE\x00\x00\x00\x00\x00\tlocalhost\x00\x00\xe1\t', + 2, + FindCoordinatorResponse[2](throttle_time_ms=0, error_code=0, error_message='NONE', node_id=0, host='localhost', port=57609), + ), + }, + ), +} + + +TEST_JOIN_GROUP = { + 'client_id': 'consumer_thread-0', + 'messages': ( + { + 'request': ( + JoinGroupRequest[5](group_id='test-group-IrXFAX', session_timeout_ms=10000, rebalance_timeout_ms=300000, member_id='', group_instance_id=None, protocol_type='consumer', protocols=[('range', b'\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00'), ('roundrobin', b'\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00')]), + b"\x00\x00\x00\xa5\x00\x0b\x00\x05\x00\x00\x00\x01\x00\x11consumer_thread-0\x00\x11test-group-IrXFAX\x00\x00'\x10\x00\x04\x93\xe0\x00\x00\xff\xff\x00\x08consumer\x00\x00\x00\x02\x00\x05range\x00\x00\x00!\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00\x00\nroundrobin\x00\x00\x00!\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00", + ), + 'response': ( + b'\x00\x00\x00N\x00\x00\x00\x01\x00\x00\x00\x00\x00O\xff\xff\xff\xff\x00\x00\x00\x00\x006consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b\x00\x00\x00\x00', + 1, + JoinGroupResponse[5](throttle_time_ms=0, error_code=79, generation_id=-1, protocol_name='', leader='', member_id='consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b', members=[]), + ), + }, + + { + 'request': ( + JoinGroupRequest[5](group_id='test-group-IrXFAX', session_timeout_ms=10000, rebalance_timeout_ms=300000, member_id='consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b', group_instance_id=None, protocol_type='consumer', protocols=[('range', b'\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00'), ('roundrobin', b'\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00')]), + b"\x00\x00\x00\xdb\x00\x0b\x00\x05\x00\x00\x00\x02\x00\x11consumer_thread-0\x00\x11test-group-IrXFAX\x00\x00'\x10\x00\x04\x93\xe0\x006consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b\xff\xff\x00\x08consumer\x00\x00\x00\x02\x00\x05range\x00\x00\x00!\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00\x00\nroundrobin\x00\x00\x00!\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00", + ), + 'response': ( + b'\x00\x00\x02\x05\x00\x00\x00\x02\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x05range\x006consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b\x006consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b\x00\x00\x00\x04\x006consumer_thread-1-a1e534ee-01a9-4c45-9dfd-778bbac550b7\xff\xff\x00\x00\x00!\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00\x006consumer_thread-3-5c966b06-f0da-4ba9-9d89-38e9da865b8c\xff\xff\x00\x00\x00!\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00\x006consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b\xff\xff\x00\x00\x00!\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00\x006consumer_thread-2-01a02bdb-5425-4bfd-979f-ddff280cecd8\xff\xff\x00\x00\x00!\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00', + 2, + JoinGroupResponse[5](throttle_time_ms=0, error_code=0, generation_id=1, protocol_name='range', leader='consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b', member_id='consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b', members=[('consumer_thread-1-a1e534ee-01a9-4c45-9dfd-778bbac550b7', None, b'\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00'), ('consumer_thread-3-5c966b06-f0da-4ba9-9d89-38e9da865b8c', None, b'\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00'), ('consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b', None, b'\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00'), ('consumer_thread-2-01a02bdb-5425-4bfd-979f-ddff280cecd8', None, b'\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x00')]), + ), + }, + + { + 'request': ( + SyncGroupRequest[3](group_id='test-group-IrXFAX', generation_id=1, member_id='consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b', group_instance_id=None, assignments=[('consumer_thread-1-a1e534ee-01a9-4c45-9dfd-778bbac550b7', b'\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x01\x00\x00\x00\x01\x00\x00\x00\x00'), ('consumer_thread-3-5c966b06-f0da-4ba9-9d89-38e9da865b8c', b'\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x01\x00\x00\x00\x03\x00\x00\x00\x00'), ('consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b', b'\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x01\x00\x00\x00\x00\x00\x00\x00\x00'), ('consumer_thread-2-01a02bdb-5425-4bfd-979f-ddff280cecd8', b'\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x01\x00\x00\x00\x02\x00\x00\x00\x00')]), + b'\x00\x00\x02\x04\x00\x0e\x00\x03\x00\x00\x00\x03\x00\x11consumer_thread-0\x00\x11test-group-IrXFAX\x00\x00\x00\x01\x006consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b\xff\xff\x00\x00\x00\x04\x006consumer_thread-1-a1e534ee-01a9-4c45-9dfd-778bbac550b7\x00\x00\x00)\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x01\x00\x00\x00\x01\x00\x00\x00\x00\x006consumer_thread-3-5c966b06-f0da-4ba9-9d89-38e9da865b8c\x00\x00\x00)\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x01\x00\x00\x00\x03\x00\x00\x00\x00\x006consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b\x00\x00\x00)\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x01\x00\x00\x00\x00\x00\x00\x00\x00\x006consumer_thread-2-01a02bdb-5425-4bfd-979f-ddff280cecd8\x00\x00\x00)\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x01\x00\x00\x00\x02\x00\x00\x00\x00', + ), + 'response': ( + b'\x00\x00\x007\x00\x00\x00\x03\x00\x00\x00\x00\x00\x00\x00\x00\x00)\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x01\x00\x00\x00\x00\x00\x00\x00\x00', + 3, + SyncGroupResponse[3](throttle_time_ms=0, error_code=0, assignment=b'\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x01\x00\x00\x00\x00\x00\x00\x00\x00'), + ), + }, + + { + 'request': ( + OffsetFetchRequest[5](group_id='test-group-IrXFAX', topics=[('test_group_fNFDXXKzKt', [0])]), + b'\x00\x00\x00Q\x00\t\x00\x05\x00\x00\x00\x04\x00\x11consumer_thread-0\x00\x11test-group-IrXFAX\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x01\x00\x00\x00\x00', + ), + 'response': ( + b'\x00\x00\x00=\x00\x00\x00\x04\x00\x00\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x01\x00\x00\x00\x00\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\x00\x00\x00\x00\x00\x00', + 4, + OffsetFetchResponse[5](throttle_time_ms=0, topics=[('test_group_fNFDXXKzKt', [(0, -1, -1, '', 0)])], error_code=0), + ), + }, + + { + 'request': ( + HeartbeatRequest[3](group_id='test-group-IrXFAX', generation_id=1, member_id='consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b', group_instance_id=None), + b'\x00\x00\x00l\x00\x0c\x00\x03\x00\x00\x00\x05\x00\x11consumer_thread-0\x00\x11test-group-IrXFAX\x00\x00\x00\x01\x006consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b\xff\xff', + ), + 'response': ( + b'\x00\x00\x00\n\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00', + 5, + HeartbeatResponse[3](throttle_time_ms=0, error_code=0), + ), + }, + + { + 'request': ( + OffsetCommitRequest[7](group_id='test-group-IrXFAX', generation_id_or_member_epoch=1, member_id='consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b', group_instance_id=None, topics=[('test_group_fNFDXXKzKt', [(0, 0, -1, '')])]), + b'\x00\x00\x00\x9d\x00\x08\x00\x07\x00\x00\x00\x06\x00\x11consumer_thread-0\x00\x11test-group-IrXFAX\x00\x00\x00\x01\x006consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b\xff\xff\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\xff\xff\xff\xff\x00\x00', + ), + 'response': ( + b'\x00\x00\x00-\x00\x00\x00\x06\x00\x00\x00\x00\x00\x00\x00\x01\x00\x15test_group_fNFDXXKzKt\x00\x00\x00\x01\x00\x00\x00\x00\x00\x00', + 6, + OffsetCommitResponse[7](throttle_time_ms=0, topics=[('test_group_fNFDXXKzKt', [(0, 0)])]), + ), + }, + + { + 'request': ( + LeaveGroupRequest[3](group_id='test-group-IrXFAX', members=[('consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b', None)]), + b'\x00\x00\x00l\x00\r\x00\x03\x00\x00\x00\x07\x00\x11consumer_thread-0\x00\x11test-group-IrXFAX\x00\x00\x00\x01\x006consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b\xff\xff', + ), + 'response': ( + b'\x00\x00\x00J\x00\x00\x00\x07\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x006consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b\xff\xff\x00\x00', + 7, + LeaveGroupResponse[3](throttle_time_ms=0, error_code=0, members=[('consumer_thread-0-b3c8845c-6071-48bc-9cf1-372f84c4409b', None, 0)]), + ), + }, + ), +} + + +TEST_CASES = [ + TEST_API_VERSIONS_1, + TEST_API_VERSIONS_2, + TEST_FIND_COORDINATOR, + TEST_JOIN_GROUP, +] + + +@pytest.mark.parametrize('test_case', TEST_CASES) +def test_parser(test_case): + parser = KafkaProtocol(client_id=test_case['client_id']) + + for msg in test_case['messages']: + req, snd = msg['request'] + + parser.send_request(req) + sent_bytes = parser.send_bytes() + assert sent_bytes == snd + + recv, correlation_id, resp = msg['response'] + + responses = parser.receive_bytes(recv) + assert len(responses) == 1 + assert responses[0][0] == correlation_id + assert responses[0][1] == resp diff --git a/test/protocol/new/test_new_sasl.py b/test/protocol/new/test_new_sasl.py new file mode 100644 index 000000000..4520299d4 --- /dev/null +++ b/test/protocol/new/test_new_sasl.py @@ -0,0 +1,50 @@ +import pytest + +from kafka.protocol.new.sasl import ( + SaslHandshakeRequest, SaslHandshakeResponse, + SaslAuthenticateRequest, SaslAuthenticateResponse, +) + + +@pytest.mark.parametrize("version", range(SaslHandshakeRequest.min_version, SaslHandshakeRequest.max_version + 1)) +def test_sasl_handshake_request_roundtrip(version): + data = SaslHandshakeRequest( + mechanism="PLAIN" + ) + encoded = SaslHandshakeRequest.encode(data, version=version) + decoded = SaslHandshakeRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(SaslHandshakeResponse.min_version, SaslHandshakeResponse.max_version + 1)) +def test_sasl_handshake_response_roundtrip(version): + data = SaslHandshakeResponse( + error_code=0, + mechanisms=["PLAIN", "SCRAM-SHA-256"] + ) + encoded = SaslHandshakeResponse.encode(data, version=version) + decoded = SaslHandshakeResponse.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(SaslAuthenticateRequest.min_version, SaslAuthenticateRequest.max_version + 1)) +def test_sasl_authenticate_request_roundtrip(version): + data = SaslAuthenticateRequest( + auth_bytes=b"sasl-payload" + ) + encoded = SaslAuthenticateRequest.encode(data, version=version) + decoded = SaslAuthenticateRequest.decode(encoded, version=version) + assert decoded == data + + +@pytest.mark.parametrize("version", range(SaslAuthenticateResponse.min_version, SaslAuthenticateResponse.max_version + 1)) +def test_sasl_authenticate_response_roundtrip(version): + data = SaslAuthenticateResponse( + error_code=0, + error_message=None, + auth_bytes=b"server-payload", + session_lifetime_ms=3600000 if version >= 1 else 0 + ) + encoded = SaslAuthenticateResponse.encode(data, version=version) + decoded = SaslAuthenticateResponse.decode(encoded, version=version) + assert decoded == data diff --git a/test/protocol/test_api_versions.py b/test/protocol/test_api_versions.py index 689357b0b..da17b399c 100644 --- a/test/protocol/test_api_versions.py +++ b/test/protocol/test_api_versions.py @@ -5,7 +5,6 @@ from kafka.protocol.api import Request, ResponseHeader from kafka.protocol.api_versions import ApiVersionsRequest, ApiVersionsResponse from kafka.protocol.types import Int32 -from kafka.version import __version__ TEST_CASES = [ @@ -25,12 +24,12 @@ ), ( - ApiVersionsRequest[3](client_software_name='kafka-python', client_software_version=__version__), + ApiVersionsRequest[3](client_software_name='kafka-python', client_software_version='2.3.0'), b'\x00\x00\x004\x00\x12\x00\x03\x00\x00\x00\x01\x00\x15_internal_client_kYVL\x00\rkafka-python\x062.3.0\x00', ), ( - ApiVersionsRequest[4](client_software_name='kafka-python', client_software_version=__version__), + ApiVersionsRequest[4](client_software_name='kafka-python', client_software_version='2.3.0'), b'\x00\x00\x004\x00\x12\x00\x04\x00\x00\x00\x01\x00\x15_internal_client_kYVL\x00\rkafka-python\x062.3.0\x00', ), diff --git a/test/protocol/test_parser.py b/test/protocol/test_parser.py index 55a4b4f1d..7397c5301 100644 --- a/test/protocol/test_parser.py +++ b/test/protocol/test_parser.py @@ -7,7 +7,6 @@ from kafka.protocol.find_coordinator import FindCoordinatorRequest, FindCoordinatorResponse from kafka.protocol.group import JoinGroupRequest, JoinGroupResponse, SyncGroupRequest, SyncGroupResponse, LeaveGroupRequest, LeaveGroupResponse, HeartbeatRequest, HeartbeatResponse from kafka.protocol.metadata import MetadataRequest, MetadataResponse -from kafka.version import __version__ TEST_API_VERSIONS_1 = { @@ -15,7 +14,7 @@ 'messages': ( { 'request': ( - ApiVersionsRequest[4](client_software_name='kafka-python', client_software_version=__version__), + ApiVersionsRequest[4](client_software_name='kafka-python', client_software_version='2.3.0'), b'\x00\x00\x004\x00\x12\x00\x04\x00\x00\x00\x01\x00\x15_internal_client_kYVL\x00\rkafka-python\x062.3.0\x00', ), 'response': ( @@ -26,7 +25,7 @@ }, { 'request': ( - ApiVersionsRequest[3](client_software_name='kafka-python', client_software_version=__version__), + ApiVersionsRequest[3](client_software_name='kafka-python', client_software_version='2.3.0'), b'\x00\x00\x004\x00\x12\x00\x03\x00\x00\x00\x02\x00\x15_internal_client_kYVL\x00\rkafka-python\x062.3.0\x00', ), 'response': ( @@ -44,7 +43,7 @@ 'messages': ( { 'request': ( - ApiVersionsRequest[4](client_software_name='kafka-python', client_software_version=__version__), + ApiVersionsRequest[4](client_software_name='kafka-python', client_software_version='2.3.0'), b'\x00\x00\x000\x00\x12\x00\x04\x00\x00\x00\x01\x00\x11admin_client_VIsf\x00\rkafka-python\x062.3.0\x00', ), 'response': ( @@ -55,7 +54,7 @@ }, { 'request': ( - ApiVersionsRequest[3](client_software_name='kafka-python', client_software_version=__version__), + ApiVersionsRequest[3](client_software_name='kafka-python', client_software_version='2.3.0'), b'\x00\x00\x000\x00\x12\x00\x03\x00\x00\x00\x02\x00\x11admin_client_VIsf\x00\rkafka-python\x062.3.0\x00', ), 'response': (