Use Fetch/Produce API v2 for brokers >= 0.10 (uses message format v1) (#694)
This commit is contained in:
@@ -581,7 +581,12 @@ class Fetcher(six.Iterator):
|
|||||||
log.debug("Adding fetch request for partition %s at offset %d",
|
log.debug("Adding fetch request for partition %s at offset %d",
|
||||||
partition, position)
|
partition, position)
|
||||||
|
|
||||||
version = 1 if self.config['api_version'] >= (0, 9) else 0
|
if self.config['api_version'] >= (0, 10):
|
||||||
|
version = 2
|
||||||
|
elif self.config['api_version'] == (0, 9):
|
||||||
|
version = 1
|
||||||
|
else:
|
||||||
|
version = 0
|
||||||
requests = {}
|
requests = {}
|
||||||
for node_id, partition_data in six.iteritems(fetchable):
|
for node_id, partition_data in six.iteritems(fetchable):
|
||||||
requests[node_id] = FetchRequest[version](
|
requests[node_id] = FetchRequest[version](
|
||||||
|
@@ -310,6 +310,12 @@ class ClusterAuthorizationFailedError(BrokerResponseError):
|
|||||||
' use an inter-broker or administrative API.')
|
' use an inter-broker or administrative API.')
|
||||||
|
|
||||||
|
|
||||||
|
class InvalidTimestampError(BrokerResponseError):
|
||||||
|
errno = 32
|
||||||
|
message = 'INVALID_TIMESTAMP'
|
||||||
|
description = ('The timestamp of the message is out of acceptable range.')
|
||||||
|
|
||||||
|
|
||||||
class KafkaUnavailableError(KafkaError):
|
class KafkaUnavailableError(KafkaError):
|
||||||
pass
|
pass
|
||||||
|
|
||||||
|
@@ -283,7 +283,8 @@ class KafkaProducer(object):
|
|||||||
if self.config['compression_type'] == 'lz4':
|
if self.config['compression_type'] == 'lz4':
|
||||||
assert self.config['api_version'] >= (0, 8, 2), 'LZ4 Requires >= Kafka 0.8.2 Brokers'
|
assert self.config['api_version'] >= (0, 8, 2), 'LZ4 Requires >= Kafka 0.8.2 Brokers'
|
||||||
|
|
||||||
self._accumulator = RecordAccumulator(**self.config)
|
message_version = 1 if self.config['api_version'] >= (0, 10) else 0
|
||||||
|
self._accumulator = RecordAccumulator(message_version=message_version, **self.config)
|
||||||
self._metadata = client.cluster
|
self._metadata = client.cluster
|
||||||
self._sender = Sender(client, self._metadata, self._accumulator,
|
self._sender = Sender(client, self._metadata, self._accumulator,
|
||||||
**self.config)
|
**self.config)
|
||||||
|
@@ -174,11 +174,16 @@ class Sender(threading.Thread):
|
|||||||
for batch in batches])
|
for batch in batches])
|
||||||
|
|
||||||
for topic, partitions in response.topics:
|
for topic, partitions in response.topics:
|
||||||
for partition, error_code, offset in partitions:
|
for partition_info in partitions:
|
||||||
|
if response.API_VERSION < 2:
|
||||||
|
partition, error_code, offset = partition_info
|
||||||
|
ts = None
|
||||||
|
else:
|
||||||
|
partition, error_code, offset, ts = partition_info
|
||||||
tp = TopicPartition(topic, partition)
|
tp = TopicPartition(topic, partition)
|
||||||
error = Errors.for_code(error_code)
|
error = Errors.for_code(error_code)
|
||||||
batch = batches_by_partition[tp]
|
batch = batches_by_partition[tp]
|
||||||
self._complete_batch(batch, error, offset)
|
self._complete_batch(batch, error, offset, ts)
|
||||||
|
|
||||||
else:
|
else:
|
||||||
# this is the acks = 0 case, just complete all requests
|
# this is the acks = 0 case, just complete all requests
|
||||||
@@ -258,7 +263,12 @@ class Sender(threading.Thread):
|
|||||||
buf = batch.records.buffer()
|
buf = batch.records.buffer()
|
||||||
produce_records_by_partition[topic][partition] = buf
|
produce_records_by_partition[topic][partition] = buf
|
||||||
|
|
||||||
version = 1 if self.config['api_version'] >= (0, 9) else 0
|
if self.config['api_version'] >= (0, 10):
|
||||||
|
version = 2
|
||||||
|
elif self.config['api_version'] == (0, 9):
|
||||||
|
version = 1
|
||||||
|
else:
|
||||||
|
version = 0
|
||||||
return ProduceRequest[version](
|
return ProduceRequest[version](
|
||||||
required_acks=acks,
|
required_acks=acks,
|
||||||
timeout=timeout,
|
timeout=timeout,
|
||||||
|
@@ -32,6 +32,12 @@ class FetchResponse_v1(Struct):
|
|||||||
)
|
)
|
||||||
|
|
||||||
|
|
||||||
|
class FetchResponse_v2(Struct):
|
||||||
|
API_KEY = 1
|
||||||
|
API_VERSION = 2
|
||||||
|
SCHEMA = FetchResponse_v1.SCHEMA # message format changed internally
|
||||||
|
|
||||||
|
|
||||||
class FetchRequest_v0(Struct):
|
class FetchRequest_v0(Struct):
|
||||||
API_KEY = 1
|
API_KEY = 1
|
||||||
API_VERSION = 0
|
API_VERSION = 0
|
||||||
@@ -56,5 +62,12 @@ class FetchRequest_v1(Struct):
|
|||||||
SCHEMA = FetchRequest_v0.SCHEMA
|
SCHEMA = FetchRequest_v0.SCHEMA
|
||||||
|
|
||||||
|
|
||||||
FetchRequest = [FetchRequest_v0, FetchRequest_v1]
|
class FetchRequest_v2(Struct):
|
||||||
FetchResponse = [FetchResponse_v0, FetchResponse_v1]
|
API_KEY = 1
|
||||||
|
API_VERSION = 2
|
||||||
|
RESPONSE_TYPE = FetchResponse_v2
|
||||||
|
SCHEMA = FetchRequest_v1.SCHEMA
|
||||||
|
|
||||||
|
|
||||||
|
FetchRequest = [FetchRequest_v0, FetchRequest_v1, FetchRequest_v2]
|
||||||
|
FetchResponse = [FetchResponse_v0, FetchResponse_v1, FetchResponse_v2]
|
||||||
|
@@ -30,6 +30,21 @@ class ProduceResponse_v1(Struct):
|
|||||||
)
|
)
|
||||||
|
|
||||||
|
|
||||||
|
class ProduceResponse_v2(Struct):
|
||||||
|
API_KEY = 0
|
||||||
|
API_VERSION = 2
|
||||||
|
SCHEMA = Schema(
|
||||||
|
('topics', Array(
|
||||||
|
('topic', String('utf-8')),
|
||||||
|
('partitions', Array(
|
||||||
|
('partition', Int32),
|
||||||
|
('error_code', Int16),
|
||||||
|
('offset', Int64),
|
||||||
|
('timestamp', Int64))))),
|
||||||
|
('thottle_time_ms', Int32)
|
||||||
|
)
|
||||||
|
|
||||||
|
|
||||||
class ProduceRequest_v0(Struct):
|
class ProduceRequest_v0(Struct):
|
||||||
API_KEY = 0
|
API_KEY = 0
|
||||||
API_VERSION = 0
|
API_VERSION = 0
|
||||||
@@ -52,5 +67,12 @@ class ProduceRequest_v1(Struct):
|
|||||||
SCHEMA = ProduceRequest_v0.SCHEMA
|
SCHEMA = ProduceRequest_v0.SCHEMA
|
||||||
|
|
||||||
|
|
||||||
ProduceRequest = [ProduceRequest_v0, ProduceRequest_v1]
|
class ProduceRequest_v2(Struct):
|
||||||
ProduceResponse = [ProduceResponse_v0, ProduceResponse_v1]
|
API_KEY = 0
|
||||||
|
API_VERSION = 2
|
||||||
|
RESPONSE_TYPE = ProduceResponse_v2
|
||||||
|
SCHEMA = ProduceRequest_v1.SCHEMA
|
||||||
|
|
||||||
|
|
||||||
|
ProduceRequest = [ProduceRequest_v0, ProduceRequest_v1, ProduceRequest_v2]
|
||||||
|
ProduceResponse = [ProduceResponse_v0, ProduceResponse_v1, ProduceResponse_v2]
|
||||||
|
@@ -15,7 +15,7 @@ from kafka.structs import TopicPartition, OffsetAndMetadata
|
|||||||
|
|
||||||
@pytest.fixture
|
@pytest.fixture
|
||||||
def client(mocker):
|
def client(mocker):
|
||||||
return mocker.Mock(spec=KafkaClient)
|
return mocker.Mock(spec=KafkaClient(bootstrap_servers=[]))
|
||||||
|
|
||||||
|
|
||||||
@pytest.fixture
|
@pytest.fixture
|
||||||
@@ -71,6 +71,19 @@ def test_init_fetches(fetcher, mocker):
|
|||||||
assert len(ret) == len(fetch_requests)
|
assert len(ret) == len(fetch_requests)
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.mark.parametrize(("api_version", "fetch_version"), [
|
||||||
|
((0, 10), 2),
|
||||||
|
((0, 9), 1),
|
||||||
|
((0, 8), 0)
|
||||||
|
])
|
||||||
|
def test_create_fetch_requests(fetcher, mocker, api_version, fetch_version):
|
||||||
|
fetcher._client.in_flight_request_count.return_value = 0
|
||||||
|
fetcher.config['api_version'] = api_version
|
||||||
|
by_node = fetcher._create_fetch_requests()
|
||||||
|
requests = by_node.values()
|
||||||
|
assert all([isinstance(r, FetchRequest[fetch_version]) for r in requests])
|
||||||
|
|
||||||
|
|
||||||
def test_update_fetch_positions(fetcher, mocker):
|
def test_update_fetch_positions(fetcher, mocker):
|
||||||
mocker.patch.object(fetcher, '_reset_offset')
|
mocker.patch.object(fetcher, '_reset_offset')
|
||||||
partition = TopicPartition('foobar', 0)
|
partition = TopicPartition('foobar', 0)
|
||||||
|
47
test/test_sender.py
Normal file
47
test/test_sender.py
Normal file
@@ -0,0 +1,47 @@
|
|||||||
|
# pylint: skip-file
|
||||||
|
from __future__ import absolute_import
|
||||||
|
|
||||||
|
import io
|
||||||
|
|
||||||
|
import pytest
|
||||||
|
|
||||||
|
from kafka.client_async import KafkaClient
|
||||||
|
from kafka.cluster import ClusterMetadata
|
||||||
|
from kafka.producer.buffer import MessageSetBuffer
|
||||||
|
from kafka.producer.sender import Sender
|
||||||
|
from kafka.producer.record_accumulator import RecordAccumulator, RecordBatch
|
||||||
|
import kafka.errors as Errors
|
||||||
|
from kafka.future import Future
|
||||||
|
from kafka.protocol.produce import ProduceRequest
|
||||||
|
from kafka.structs import TopicPartition, OffsetAndMetadata
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.fixture
|
||||||
|
def client(mocker):
|
||||||
|
_cli = mocker.Mock(spec=KafkaClient(bootstrap_servers=[]))
|
||||||
|
_cli.cluster = mocker.Mock(spec=ClusterMetadata())
|
||||||
|
return _cli
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.fixture
|
||||||
|
def accumulator():
|
||||||
|
return RecordAccumulator()
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.fixture
|
||||||
|
def sender(client, accumulator):
|
||||||
|
return Sender(client, client.cluster, accumulator)
|
||||||
|
|
||||||
|
|
||||||
|
@pytest.mark.parametrize(("api_version", "produce_version"), [
|
||||||
|
((0, 10), 2),
|
||||||
|
((0, 9), 1),
|
||||||
|
((0, 8), 0)
|
||||||
|
])
|
||||||
|
def test_produce_request(sender, mocker, api_version, produce_version):
|
||||||
|
sender.config['api_version'] = api_version
|
||||||
|
tp = TopicPartition('foo', 0)
|
||||||
|
records = MessageSetBuffer(io.BytesIO(), 100000)
|
||||||
|
batch = RecordBatch(tp, records)
|
||||||
|
produce_request = sender._produce_request(0, 0, 0, [batch])
|
||||||
|
assert isinstance(produce_request, ProduceRequest[produce_version])
|
Reference in New Issue
Block a user