You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 15 Next »

Status

Current state: Accepted

Discussion thread: here

JIRA: KAFKA-7632

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

Basically, CPU (running time) and I/O (compressed size) are trade-offs in compression. Since the best is use case dependent, lots of compression libraries provide a way to control the compression level with a reasonable default level, which results in a good performance in general. However, Kafka does not provide a way to configure the compression level - it uses the default level only.

This proposal suggests adding the compression level option to the producer, broker, and topic config. Running tests with a real-world dataset (see below), I found that this option improves the producer's message/second rate up to 156%.

Public Interfaces

This feature introduces a new option, 'compression.level', to the producer, topic, and broker configuration. The type of this option is an integer, with a default value of null.

compression.type=gzip
compression.level=4				# NEW: Compression level to be used.

The table below shows the valid range of compression.level per compression.type. (note: snappy is excluded since it does not support any compression level.) The valid range and default value of the compression level are entirely up to the compression library, so they may be changed in the future.

Compression CodecavailabilityValid RangeDefault
gzipYes1 (Deflater.BEST_SPEED) ~ 9 (Deflater.BEST_COMPRESSION)6
snappyNo--
lz4Yes1 ~ 179
zstdYes-131072 ~ 223

Proposed Changes

This option impacts the following processes:

  • Producer compresses the user-given messages.
  • Broker recompresses the user-given messages with specified compression.type per broker or topic.
  • Broker recompresses the messages in the log cleanup process.

Compressing the records with the given compression type and level works like the following:

  • If 'compression.type' is none of snappy, 'compression.level' is ignored.
  • If 'compression.level' is not in the valid range, it raises an error.
  • If 'compression.level' is in the valid range, the producer compresses the records with the given level.
  • If 'compression.level' is not set, it falls back to the default level.

Benchmark

Produce Test

To benchmark how compression level affects the producer performance, I ran a small benchmark with a real-world dataset like below:

Producer

With the feature implemented on top of the latest trunk (commit ccec9b0), I ran kafka-producer-perf-test.sh on GraalVM Java 8 v21.1.0 with the following parameters:

  • Number of records: 100,000
  • batch.size: 1048576 (1mb)
  • linger.ms: 100

Data

A random sample of 4096 real-world records from this dataset, which consists of 129218 json files with an average size of 55.25kb. 

Environment

MS Azure Kubernetes Cluster (Seoul Region), consists of 16 nodes of Standard_DS2_v2 (2vCPU, 7GB RAM, Expected network bandwidth of 1500 Mbps.)

Broker/Topic

Apache Kafka 2.7.0, GraalVM Java 8 (21.1.0), replicaton factor = 3.

Result

codeclevelproduced message / seclatency (ms)size (bytes)description
none
2,739.50205.345,659,454,754
gzip11,122.961,230.221,787,505,238min. level
gzip6717.712,041.241,644,280,629default level
gzip9608.542,413.661,643,517,758max. level
lz411,694.69603.462,211,346,795min. level
lz491,199.93878.852,184,022,257default level
lz417495.342,110.552,178,643,665max. level
zstd-57,653.45156.881,997,500,892experimental level
zstd16,317.5268.551,521,783,958
zstd34,760.54286.791,494,620,615default level
zstd12988.95863.891,458,150,768
zstd1885.202,017.921,492,015,424

It shows the following:

  • Codec is the main factor that differentiates the compressed size. However, The compression level makes little impact on it. The maximum improvement is is gzip/1 vs. gzip/9 (8%), and the minimum is lz4/1 vs. lz/17 (1.5%).
  • Excepting zstd/-5, when the compression level gets lower, messages/sec increase but latency decreases. Especially, compressing with zstd/1 produces 32.7% more messages per second than zstd/3 (current default), and gzip/1 produces 56.4% than gzip/6 (current default).
  • For every compression codec, compression with minimum level (i.e., speed first strategy) resulted in the best messages/second rate.

Linear Write Test

To benchmark how compression level affects the linear write performance, I ran a small benchmark with a real-world dataset like below:

INCLUDE_TEST_JARS=true bin/kafka-run-class.sh kafka.TestLinearWriteSpeed --bytes 8192 --size 8192 --message-size 4096 --files 1 --compression {compression-codec} --level {compression-level} --log

Result

codeclevelwrite speed (mb/sec)description
none
19678.841
gzip122007.042min. level
gzip618425.707default level
gzip919148.284max. level
lz4122776.967min. level
lz4920613.456default level
lz41719879.134max. level
zstd-519531.25experimental level
zstd122910.557
zstd319531.25default level
zstd1217477.628
zstd1821229.619

The result was almost similar. In general, the minimum compression level (=1) showed the best write speed (except zstd/-5).

Compatibility, Deprecation, and Migration Plan

Since this update follows the default compression level and current buffer size if they are not set, there is no backward compatibility problem.

Further works

Alongside the compression level, I am trying additional configuration options, like the following:

Compression buffer size option

At the initial stage, the compression buffer size option was also under consideration. However, during the benchmark, I could not find its positive impacts on produce speed or compressed size. I am still investigating whether it can improve the disk write speed, e.g., Broker-size recompression or compaction.

Long window size with Zstandard

With 1.3.2, Zstandard introduced compression/decompression with long window size. This option can improve the compression/decompression speed for some levels.

Rejected Alternatives

None.

  • No labels