Skip to content

Commit 5d30e41

Browse files
author
Tincu Gabriel
committed
Add support for zstd compression (dpkp#2021)
1 parent f73cabc commit 5d30e41

File tree

6 files changed

+21
-17
lines changed

6 files changed

+21
-17
lines changed

.travis.yml

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ addons:
2222
apt:
2323
packages:
2424
- libsnappy-dev
25+
- libzstd-dev
2526
- openjdk-8-jdk
2627

2728
cache:

docs/index.rst

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -122,11 +122,12 @@ multiprocessing is recommended.
122122
Compression
123123
***********
124124

125-
kafka-python supports gzip compression/decompression natively. To produce or
126-
consume lz4 compressed messages, you should install python-lz4 (pip install lz4).
127-
To enable snappy, install python-snappy (also requires snappy library).
128-
See `Installation <install.html#optional-snappy-install>`_ for more information.
125+
kafka-python supports multiple compression types:
129126

127+
- gzip : supported natively
128+
- lz4 : requires `python-lz4 <https://pypi.org/project/lz4/>`_ installed
129+
- snappy : requires the `python-snappy <https://pypi.org/project/python-snappy/>`_ package (which requires the snappy C library)
130+
- zstd : requires the `python-zstandard <https://github.com/indygreg/python-zstandard>`_ package installed
130131

131132
Protocol
132133
********

kafka/codec.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,7 @@
1010

1111
_XERIAL_V1_HEADER = (-126, b'S', b'N', b'A', b'P', b'P', b'Y', 0, 1, 1)
1212
_XERIAL_V1_FORMAT = 'bccccccBii'
13-
ZSTD_MAX_OUTPUT_SIZE = 1024 ** 3
13+
ZSTD_MAX_OUTPUT_SIZE = 1024 * 1024
1414

1515
try:
1616
import snappy

kafka/protocol/message.py

Lines changed: 7 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -3,8 +3,8 @@
33
import io
44
import time
55

6-
from kafka.codec import (has_gzip, has_snappy, has_lz4,
7-
gzip_decode, snappy_decode,
6+
from kafka.codec import (has_gzip, has_snappy, has_lz4, has_zstd,
7+
gzip_decode, snappy_decode, zstd_decode,
88
lz4_decode, lz4_decode_old_kafka)
99
from kafka.protocol.frame import KafkaBytes
1010
from kafka.protocol.struct import Struct
@@ -35,6 +35,7 @@ class Message(Struct):
3535
CODEC_GZIP = 0x01
3636
CODEC_SNAPPY = 0x02
3737
CODEC_LZ4 = 0x03
38+
CODEC_ZSTD = 0x04
3839
TIMESTAMP_TYPE_MASK = 0x08
3940
HEADER_SIZE = 22 # crc(4), magic(1), attributes(1), timestamp(8), key+value size(4*2)
4041

@@ -119,7 +120,7 @@ def is_compressed(self):
119120

120121
def decompress(self):
121122
codec = self.attributes & self.CODEC_MASK
122-
assert codec in (self.CODEC_GZIP, self.CODEC_SNAPPY, self.CODEC_LZ4)
123+
assert codec in (self.CODEC_GZIP, self.CODEC_SNAPPY, self.CODEC_LZ4, self.CODEC_ZSTD)
123124
if codec == self.CODEC_GZIP:
124125
assert has_gzip(), 'Gzip decompression unsupported'
125126
raw_bytes = gzip_decode(self.value)
@@ -132,6 +133,9 @@ def decompress(self):
132133
raw_bytes = lz4_decode_old_kafka(self.value)
133134
else:
134135
raw_bytes = lz4_decode(self.value)
136+
elif codec == self.CODEC_ZSTD:
137+
assert has_zstd(), "ZSTD decompression unsupported"
138+
raw_bytes = zstd_decode(self.value)
135139
else:
136140
raise Exception('This should be impossible')
137141

test/test_codec.py

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -22,15 +22,15 @@ def test_gzip():
2222
for i in range(1000):
2323
b1 = random_string(100).encode('utf-8')
2424
b2 = gzip_decode(gzip_encode(b1))
25-
assert b1 == b2, "decompressed value differs from input value: %r vs %r" % (b2, b1)
25+
assert b1 == b2
2626

2727

2828
@pytest.mark.skipif(not has_snappy(), reason="Snappy not available")
2929
def test_snappy():
3030
for i in range(1000):
3131
b1 = random_string(100).encode('utf-8')
3232
b2 = snappy_decode(snappy_encode(b1))
33-
assert b1 == b2, "decompressed value differs from input value: %r vs %r" % (b2, b1)
33+
assert b1 == b2
3434

3535

3636
@pytest.mark.skipif(not has_snappy(), reason="Snappy not available")
@@ -91,7 +91,7 @@ def test_lz4():
9191
b1 = random_string(100).encode('utf-8')
9292
b2 = lz4_decode(lz4_encode(b1))
9393
assert len(b1) == len(b2)
94-
assert b1 == b2, "decompressed value differs from input value: %r vs %r" % (b2, b1)
94+
assert b1 == b2
9595

9696

9797
@pytest.mark.skipif(not has_lz4() or platform.python_implementation() == 'PyPy',
@@ -101,7 +101,7 @@ def test_lz4_old():
101101
b1 = random_string(100).encode('utf-8')
102102
b2 = lz4_decode_old_kafka(lz4_encode_old_kafka(b1))
103103
assert len(b1) == len(b2)
104-
assert b1 == b2, "decompressed value differs from input value: %r vs %r" % (b2, b1)
104+
assert b1 == b2
105105

106106

107107
@pytest.mark.skipif(not has_lz4() or platform.python_implementation() == 'PyPy',
@@ -113,12 +113,12 @@ def test_lz4_incremental():
113113
b1 = random_string(100).encode('utf-8') * 50000
114114
b2 = lz4_decode(lz4_encode(b1))
115115
assert len(b1) == len(b2)
116-
assert b1 == b2, "decompressed value differs from input value: %r vs %r" % (b2, b1)
116+
assert b1 == b2
117117

118118

119119
@pytest.mark.skipif(not has_zstd(), reason="Zstd not available")
120120
def test_zstd():
121121
for _ in range(1000):
122122
b1 = random_string(100).encode('utf-8')
123123
b2 = zstd_decode(zstd_encode(b1))
124-
assert b1 == b2, "decompressed value differs from input value: %r vs %r" % (b2, b1)
124+
assert b1 == b2

test/test_producer.py

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -32,13 +32,12 @@ def test_end_to_end(kafka_broker, compression):
3232
pytest.skip('python-lz4 crashes on older versions of pypy')
3333

3434
if compression == 'zstd' and env_kafka_version() < (2, 1, 0):
35-
pytest.skip("zstd requires kafka 2.1.0 or more")
35+
pytest.skip('zstd requires kafka 2.1.0 or newer')
3636

3737
connect_str = ':'.join([kafka_broker.host, str(kafka_broker.port)])
3838
producer = KafkaProducer(bootstrap_servers=connect_str,
3939
retries=5,
4040
max_block_ms=30000,
41-
api_version=env_kafka_version(),
4241
compression_type=compression,
4342
value_serializer=str.encode)
4443
consumer = KafkaConsumer(bootstrap_servers=connect_str,
@@ -89,7 +88,6 @@ def test_kafka_producer_proper_record_metadata(kafka_broker, compression):
8988
connect_str = ':'.join([kafka_broker.host, str(kafka_broker.port)])
9089
producer = KafkaProducer(bootstrap_servers=connect_str,
9190
retries=5,
92-
api_version=env_kafka_version(),
9391
max_block_ms=30000,
9492
compression_type=compression)
9593
magic = producer._max_usable_produce_magic()

0 commit comments

Comments
 (0)