Skip to content

Commit

Permalink
Merge 6f42357 into 34dc36d
Browse files Browse the repository at this point in the history
  • Loading branch information
Tincu Gabriel committed Mar 14, 2020
2 parents 34dc36d + 6f42357 commit 0a6f949
Show file tree
Hide file tree
Showing 13 changed files with 115 additions and 23 deletions.
2 changes: 1 addition & 1 deletion build_integration.sh
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
#!/bin/bash

: ${ALL_RELEASES:="0.8.2.2 0.9.0.1 0.10.1.1 0.10.2.2 0.11.0.3 1.0.2 1.1.1 2.0.1"}
: ${ALL_RELEASES:="0.8.2.2 0.9.0.1 0.10.1.1 0.10.2.2 0.11.0.3 1.0.2 1.1.1 2.0.1 2.1.0"}
: ${SCALA_VERSION:=2.11}
: ${DIST_BASE_URL:=https://archive.apache.org/dist/kafka/}
: ${KAFKA_SRC_GIT:=https://github.com/apache/kafka.git}
Expand Down
21 changes: 21 additions & 0 deletions kafka/codec.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,11 @@
except ImportError:
snappy = None

try:
import zstandard as zstd
except ImportError:
zstd = None

try:
import lz4.frame as lz4

Expand Down Expand Up @@ -58,6 +63,10 @@ def has_snappy():
return snappy is not None


def has_zstd():
return zstd is not None


def has_lz4():
if lz4 is not None:
return True
Expand Down Expand Up @@ -299,3 +308,15 @@ def lz4_decode_old_kafka(payload):
payload[header_size:]
])
return lz4_decode(munged_payload)


def zstd_encode(payload):
if not zstd:
raise NotImplementedError("Zstd codec is not available")
return zstd.ZstdCompressor().compress(payload)


def zstd_decode(payload):
if not zstd:
raise NotImplementedError("Zstd codec is not available")
return zstd.ZstdDecompressor().decompress(payload)
6 changes: 3 additions & 3 deletions kafka/producer/future.py
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ def __init__(self, produce_future, relative_offset, timestamp_ms, checksum, seri
produce_future.add_errback(self.failure)

def _produce_success(self, offset_and_timestamp):
offset, produce_timestamp_ms = offset_and_timestamp
offset, produce_timestamp_ms, log_start_offset = offset_and_timestamp

# Unpacking from args tuple is minor speed optimization
(relative_offset, timestamp_ms, checksum,
Expand All @@ -51,7 +51,7 @@ def _produce_success(self, offset_and_timestamp):
if offset != -1 and relative_offset is not None:
offset += relative_offset
tp = self._produce_future.topic_partition
metadata = RecordMetadata(tp[0], tp[1], tp, offset, timestamp_ms,
metadata = RecordMetadata(tp[0], tp[1], tp, offset, timestamp_ms, log_start_offset,
checksum, serialized_key_size,
serialized_value_size, serialized_header_size)
self.success(metadata)
Expand All @@ -67,5 +67,5 @@ def get(self, timeout=None):


RecordMetadata = collections.namedtuple(
'RecordMetadata', ['topic', 'partition', 'topic_partition', 'offset', 'timestamp',
'RecordMetadata', ['topic', 'partition', 'topic_partition', 'offset', 'timestamp', 'log_start_offset',
'checksum', 'serialized_key_size', 'serialized_value_size', 'serialized_header_size'])
8 changes: 6 additions & 2 deletions kafka/producer/kafka.py
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@

import kafka.errors as Errors
from kafka.client_async import KafkaClient, selectors
from kafka.codec import has_gzip, has_snappy, has_lz4
from kafka.codec import has_gzip, has_snappy, has_lz4, has_zstd
from kafka.metrics import MetricConfig, Metrics
from kafka.partitioner.default import DefaultPartitioner
from kafka.producer.future import FutureRecordMetadata, FutureProduceResult
Expand Down Expand Up @@ -119,7 +119,7 @@ class KafkaProducer(object):
available guarantee.
If unset, defaults to acks=1.
compression_type (str): The compression type for all data generated by
the producer. Valid values are 'gzip', 'snappy', 'lz4', or None.
the producer. Valid values are 'gzip', 'snappy', 'lz4', 'zstd' or None.
Compression is of full batches of data, so the efficacy of batching
will also impact the compression ratio (more batching means better
compression). Default: None.
Expand Down Expand Up @@ -339,6 +339,7 @@ class KafkaProducer(object):
'gzip': (has_gzip, LegacyRecordBatchBuilder.CODEC_GZIP),
'snappy': (has_snappy, LegacyRecordBatchBuilder.CODEC_SNAPPY),
'lz4': (has_lz4, LegacyRecordBatchBuilder.CODEC_LZ4),
'zstd': (has_zstd, DefaultRecordBatchBuilder.CODEC_ZSTD),
None: (lambda: True, LegacyRecordBatchBuilder.CODEC_NONE),
}

Expand Down Expand Up @@ -388,6 +389,9 @@ def __init__(self, **configs):
if self.config['compression_type'] == 'lz4':
assert self.config['api_version'] >= (0, 8, 2), 'LZ4 Requires >= Kafka 0.8.2 Brokers'

if self.config['compression_type'] == 'zstd':
assert self.config['api_version'] >= (2, 1, 0), 'Zstd Requires >= Kafka 2.1.0 Brokers'

# Check compression_type for library support
ct = self.config['compression_type']
if ct not in self._COMPRESSORS:
Expand Down
8 changes: 4 additions & 4 deletions kafka/producer/record_accumulator.py
Original file line number Diff line number Diff line change
Expand Up @@ -68,16 +68,16 @@ def try_append(self, timestamp_ms, key, value, headers):
sum(len(h_key.encode("utf-8")) + len(h_val) for h_key, h_val in headers) if headers else -1)
return future

def done(self, base_offset=None, timestamp_ms=None, exception=None):
def done(self, base_offset=None, timestamp_ms=None, exception=None, log_start_offset=None):
level = logging.DEBUG if exception is None else logging.WARNING
log.log(level, "Produced messages to topic-partition %s with base offset"
" %s and error %s.", self.topic_partition, base_offset,
exception) # trace
" %s log start offset %s and error %s.", self.topic_partition, base_offset,
log_start_offset, exception) # trace
if self.produce_future.is_done:
log.warning('Batch is already closed -- ignoring batch.done()')
return
elif exception is None:
self.produce_future.success((base_offset, timestamp_ms))
self.produce_future.success((base_offset, timestamp_ms, log_start_offset))
else:
self.produce_future.failure(exception)

Expand Down
13 changes: 9 additions & 4 deletions kafka/producer/sender.py
Original file line number Diff line number Diff line change
Expand Up @@ -198,8 +198,10 @@ def _handle_produce_response(self, node_id, send_time, batches, response):
if response.API_VERSION < 2:
partition, error_code, offset = partition_info
ts = None
else:
elif 3 <= response.API_VERSION <= 4:
partition, error_code, offset, ts = partition_info
elif 5 <= response.API_VERSION <= 7:
partition, error_code, offset, ts, log_start_offset = partition_info
tp = TopicPartition(topic, partition)
error = Errors.for_code(error_code)
batch = batches_by_partition[tp]
Expand All @@ -213,14 +215,15 @@ def _handle_produce_response(self, node_id, send_time, batches, response):
for batch in batches:
self._complete_batch(batch, None, -1, None)

def _complete_batch(self, batch, error, base_offset, timestamp_ms=None):
def _complete_batch(self, batch, error, base_offset, timestamp_ms=None, log_start_offset=None):
"""Complete or retry the given batch of records.
Arguments:
batch (RecordBatch): The record batch
error (Exception): The error (or None if none)
base_offset (int): The base offset assigned to the records if successful
timestamp_ms (int, optional): The timestamp returned by the broker for this batch
log_start_offset (int): The start offset of the log at the time this produce response was created
"""
# Standardize no-error to None
if error is Errors.NoError:
Expand All @@ -240,7 +243,7 @@ def _complete_batch(self, batch, error, base_offset, timestamp_ms=None):
error = error(batch.topic_partition.topic)

# tell the user the result of their request
batch.done(base_offset, timestamp_ms, error)
batch.done(base_offset, timestamp_ms, error, log_start_offset)
self._accumulator.deallocate(batch)
if error is not None:
self._sensors.record_errors(batch.topic_partition.topic, batch.record_count)
Expand Down Expand Up @@ -293,7 +296,9 @@ def _produce_request(self, node_id, acks, timeout, batches):
produce_records_by_partition[topic][partition] = buf

kwargs = {}
if self.config['api_version'] >= (0, 11):
if self.config['api_version'] >= (2, 1):
version = 7
elif self.config['api_version'] >= (0, 11):
version = 3
kwargs = dict(transactional_id=None)
elif self.config['api_version'] >= (0, 10):
Expand Down
44 changes: 41 additions & 3 deletions kafka/protocol/produce.py
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,6 @@ class ProduceResponse_v4(Response):
API_VERSION = 4
SCHEMA = ProduceResponse_v3.SCHEMA


class ProduceResponse_v5(Response):
API_KEY = 0
API_VERSION = 5
Expand All @@ -78,6 +77,25 @@ class ProduceResponse_v5(Response):
)


class ProduceResponse_v6(Response):
"""
Same as V5
"""
API_KEY = 0
API_VERSION = 6
SCHEMA = ProduceResponse_v5.SCHEMA


class ProduceResponse_v7(Response):
"""
Same as V6
"""
API_KEY = 0
API_VERSION = 7
SCHEMA = ProduceResponse_v6.SCHEMA



class ProduceRequest(Request):
API_KEY = 0

Expand Down Expand Up @@ -147,11 +165,31 @@ class ProduceRequest_v5(ProduceRequest):
SCHEMA = ProduceRequest_v4.SCHEMA


class ProduceRequest_v6(ProduceRequest):
"""
Same as V5
"""
API_VERSION = 6
RESPONSE_TYPE = ProduceResponse_v6
SCHEMA = ProduceRequest_v5.SCHEMA


class ProduceRequest_v7(ProduceRequest):
"""
Same as V6
"""
API_VERSION = 7
RESPONSE_TYPE = ProduceResponse_v7
SCHEMA = ProduceRequest_v6.SCHEMA


ProduceRequest = [
ProduceRequest_v0, ProduceRequest_v1, ProduceRequest_v2,
ProduceRequest_v3, ProduceRequest_v4, ProduceRequest_v5
ProduceRequest_v3, ProduceRequest_v4, ProduceRequest_v5,
ProduceRequest_v6, ProduceRequest_v7,
]
ProduceResponse = [
ProduceResponse_v0, ProduceResponse_v1, ProduceResponse_v2,
ProduceResponse_v3, ProduceResponse_v4, ProduceResponse_v5
ProduceResponse_v3, ProduceResponse_v4, ProduceResponse_v5,
ProduceRequest_v6, ProduceRequest_v7,
]
11 changes: 9 additions & 2 deletions kafka/record/default_records.py
Original file line number Diff line number Diff line change
Expand Up @@ -62,8 +62,8 @@
)
from kafka.errors import CorruptRecordException, UnsupportedCodecError
from kafka.codec import (
gzip_encode, snappy_encode, lz4_encode,
gzip_decode, snappy_decode, lz4_decode
gzip_encode, snappy_encode, lz4_encode, zstd_encode,
gzip_decode, snappy_decode, lz4_decode, zstd_decode
)
import kafka.codec as codecs

Expand Down Expand Up @@ -97,6 +97,7 @@ class DefaultRecordBase(object):
CODEC_GZIP = 0x01
CODEC_SNAPPY = 0x02
CODEC_LZ4 = 0x03
CODEC_ZSTD = 0x04
TIMESTAMP_TYPE_MASK = 0x08
TRANSACTIONAL_MASK = 0x10
CONTROL_MASK = 0x20
Expand All @@ -111,6 +112,8 @@ def _assert_has_codec(self, compression_type):
checker, name = codecs.has_snappy, "snappy"
elif compression_type == self.CODEC_LZ4:
checker, name = codecs.has_lz4, "lz4"
elif compression_type == self.CODEC_ZSTD:
checker, name = codecs.has_zstd, "zstd"
if not checker():
raise UnsupportedCodecError(
"Libraries for {} compression codec not found".format(name))
Expand Down Expand Up @@ -185,6 +188,8 @@ def _maybe_uncompress(self):
uncompressed = snappy_decode(data.tobytes())
if compression_type == self.CODEC_LZ4:
uncompressed = lz4_decode(data.tobytes())
if compression_type == self.CODEC_ZSTD:
uncompressed = zstd_decode(data)
self._buffer = bytearray(uncompressed)
self._pos = 0
self._decompressed = True
Expand Down Expand Up @@ -517,6 +522,8 @@ def _maybe_compress(self):
compressed = snappy_encode(data)
elif self._compression_type == self.CODEC_LZ4:
compressed = lz4_encode(data)
elif self._compression_type == self.CODEC_ZSTD:
compressed = zstd_encode(data)
compressed_size = len(compressed)
if len(data) <= compressed_size:
# We did not get any benefit from compression, lets send
Expand Down
2 changes: 1 addition & 1 deletion kafka/record/memory_records.py
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@ class MemoryRecordsBuilder(object):

def __init__(self, magic, compression_type, batch_size):
assert magic in [0, 1, 2], "Not supported magic"
assert compression_type in [0, 1, 2, 3], "Not valid compression type"
assert compression_type in [0, 1, 2, 3, 4], "Not valid compression type"
if magic >= 2:
self._builder = DefaultRecordBatchBuilder(
magic=magic, compression_type=compression_type,
Expand Down
1 change: 1 addition & 0 deletions requirements-dev.txt
Original file line number Diff line number Diff line change
Expand Up @@ -15,3 +15,4 @@ pytest-mock==1.10.0
sphinx-rtd-theme==0.2.4
crc32c==1.7
py==1.8.0
zstandard==0.13.0
11 changes: 10 additions & 1 deletion test/test_codec.py
Original file line number Diff line number Diff line change
Expand Up @@ -7,11 +7,12 @@
from kafka.vendor.six.moves import range

from kafka.codec import (
has_snappy, has_lz4,
has_snappy, has_lz4, has_zstd,
gzip_encode, gzip_decode,
snappy_encode, snappy_decode,
lz4_encode, lz4_decode,
lz4_encode_old_kafka, lz4_decode_old_kafka,
zstd_encode, zstd_decode,
)

from test.testutil import random_string
Expand Down Expand Up @@ -113,3 +114,11 @@ def test_lz4_incremental():
b2 = lz4_decode(lz4_encode(b1))
assert len(b1) == len(b2)
assert b1 == b2


@pytest.mark.skipif(not has_zstd(), reason="Zstd not available")
def test_zstd():
for _ in range(1000):
b1 = random_string(100).encode('utf-8')
b2 = zstd_decode(zstd_encode(b1))
assert b1 == b2
10 changes: 8 additions & 2 deletions test/test_producer.py
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ def test_buffer_pool():


@pytest.mark.skipif(not env_kafka_version(), reason="No KAFKA_VERSION set")
@pytest.mark.parametrize("compression", [None, 'gzip', 'snappy', 'lz4'])
@pytest.mark.parametrize("compression", [None, 'gzip', 'snappy', 'lz4', 'zstd'])
def test_end_to_end(kafka_broker, compression):

if compression == 'lz4':
Expand All @@ -34,6 +34,9 @@ def test_end_to_end(kafka_broker, compression):
elif platform.python_implementation() == 'PyPy':
return

if compression == 'zstd' and env_kafka_version() < (2, 1, 0):
return

connect_str = ':'.join([kafka_broker.host, str(kafka_broker.port)])
producer = KafkaProducer(bootstrap_servers=connect_str,
retries=5,
Expand Down Expand Up @@ -81,8 +84,10 @@ def test_kafka_producer_gc_cleanup():


@pytest.mark.skipif(not env_kafka_version(), reason="No KAFKA_VERSION set")
@pytest.mark.parametrize("compression", [None, 'gzip', 'snappy', 'lz4'])
@pytest.mark.parametrize("compression", [None, 'gzip', 'snappy', 'lz4', 'zstd'])
def test_kafka_producer_proper_record_metadata(kafka_broker, compression):
if compression == 'zstd' and env_kafka_version() < (2, 1, 0):
return
connect_str = ':'.join([kafka_broker.host, str(kafka_broker.port)])
producer = KafkaProducer(bootstrap_servers=connect_str,
retries=5,
Expand All @@ -91,6 +96,7 @@ def test_kafka_producer_proper_record_metadata(kafka_broker, compression):
magic = producer._max_usable_produce_magic()

# record headers are supported in 0.11.0

if env_kafka_version() < (0, 11, 0):
headers = None
else:
Expand Down
1 change: 1 addition & 0 deletions tox.ini
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ deps =
pytest-mock
mock
python-snappy
zstandard
lz4
xxhash
crc32c
Expand Down

0 comments on commit 0a6f949

Please sign in to comment.