If it is not compressed, you should be able to see the raw strings in the
partition data file with a text reader, such as 'less'(mixed with binary
content.

On Wed, May 12, 2021, 23:58 Shantanu Deshmukh <shantanu...@gmail.com> wrote:

> Hi Scott,
> Thanks for responding.
>
> As part of my test. I have also run my own program wherein I used both
> random strings and json messages.
> I was aware that snappy may not perform good on random strings. So I used a
> 10kb json message and sent 10,000 messages of that same json.
> So with this test payload the entire message is same, repeating over and
> over again.
>
> At least with that it could have compressed. But to no avail!
>
> I also applied batch.size=10000 and linger.ms=10 still it didnt work.
>
> On Thu, May 13, 2021 at 11:32 AM Scott Carey <scott.ca...@gmail.com>
> wrote:
>
> > Snappy and lz4 do not have entropy encoders (
> > https://en.wikipedia.org/wiki/Entropy_encoding).  If your data is random
> > text they will not compress.  If your text is a string of all zeros or
> any
> > repeating pattern, it will compress significantly.  If its something like
> > JSON, or XML it will compress.
> >
> > I suspect you either aren't using real world data, or haven't compared
> the
> > compression with different types of data (json?  web pages?   numbers?).
> > No compression test is of much use unless you specify _what_ you are
> trying
> > to compress and either construct a realistic corpus for your use case, or
> > test with a few well defined types of real data that might be similar to
> > your expected use case.
> >
> >   Gzip and zstandard have entropy encoding (Huffman for gzip, and a
> > combination of Huffman and ANS for zstandard).  With these, even if your
> > text is purely random _text_, it will compress somewhat since text
> doesn't
> > use all 256 possible byte values and so it can use less than 8 bits per
> > character in the encoding.
> >
> >
> >
> > On Wed, May 12, 2021, 22:35 Shantanu Deshmukh <shantanu...@gmail.com>
> > wrote:
> >
> > > I have some updates on this.
> > > I tried this on latest kafka 2.8. Ran my application. Results are same,
> > > snappy and lz4 dont seem to be working as uncompressed and compressed
> > > storage both measure the same.
> > >
> > > *I even tried kafka-producer-perf-test tool*. Below are the results
> > >
> > > Without any compression:
> > > ==========================>>
> > > sh bin/kafka-producer-perf-test.sh --num-records 100000 --throughput
> > 10000
> > > --record-size 102400 --topic perf-test-uncompressed --producer-props
> > > *compression.type=none* bootstrap.servers=localhost:9092
> --print-metrics
> > >
> > > 100000 records sent, *862.113558 records/sec (84.19 MB/sec)*, 376.08 ms
> > avg
> > > latency, 1083.00 ms max latency, 371 ms 50th, 610 ms 95th, 778 ms 99th,
> > > 1061 ms 99.9th.
> > > ...
> > > producer-topic-metrics:*compression-rate*:{client-id=producer-1,
> > > topic=perf-test-uncompressed}   : *1.000*
> > >
> > > With snappy compression:
> > > ==========================>>
> > > sh bin/kafka-producer-perf-test.sh --num-records 100000 --throughput
> > 10000
> > > --record-size 102400 --topic perf-test-uncompressed --producer-props
> > > *compression.type=snappy
> > > batch.size=100000 linger.ms <http://linger.ms>=5
> > > *bootstrap.servers=localhost:9092
> > > --print-metrics
> > >
> > > 100000 records sent, 599.905215 *records/sec (58.58 MB/sec)*, 540.79 ms
> > avg
> > > latency, 1395.00 ms max latency, 521 ms 50th, 816 ms 95th, 1016 ms
> 99th,
> > > 1171 ms 99.9th.
> > > ...
> > > producer-topic-metrics:*compression-rate*:{client-id=producer-1,
> > > topic=perf-test-uncompressed}   : *1.001*
> > >
> > > <<======++++===============
> > > Above mentioned compression-rate didnt change even with
> > >
> > > With  Gzip compression
> > > *==========================>>*
> > > sh bin/kafka-producer-perf-test.sh --num-records 100000 --throughput
> > 10000
> > > --record-size 102400 --topic perf-test-compressed --producer-props
> > > *compression.type=gzip* bootstrap.servers=localhost:9092
> > *batch.size=100000
> > > linger.ms <http://linger.ms>=5* --print-metrics
> > >
> > > 100000 records sent, *200.760078 records/sec (19.61 MB/sec)*, 1531.40
> ms
> > > avg latency, 2744.00 ms max latency, 1514 ms 50th, 1897 ms 95th, 2123
> ms
> > > 99th, 2610 ms 99.9th.
> > > ...
> > > producer-topic-metrics:*compression-rate*:{client-id=producer-1,
> > > topic=perf-test-compressed}   : *0.635*
> > >
> > > *<<============================*
> > >
> > > To summarise*:*
> > > compression type
> > > messages sent
> > > avg latency/throughput
> > > effective compression-rate
> > > none
> > > 100000
> > > 862.113558 records/sec (84.19 MB/sec)
> > > 1.000
> > > snappy
> > > 100000
> > > 599.905215 records/sec (58.58 MB/sec),
> > > 1.001
> > > gzip
> > > 100000
> > > 200.760078 records/sec (19.61 MB/sec)
> > > 0.635
> > >
> > > In short snappy = uncompressed !! Why is this happening?
> > >
> > > On Wed, May 12, 2021 at 11:40 AM Shantanu Deshmukh <
> > shantanu...@gmail.com>
> > > wrote:
> > >
> > > > Hey Nitin,
> > > >
> > > > I have already done that. I used dump-log-segments option. And I can
> > see
> > > > the codec used is snappy/gzip/lz4. My question is, only gzip is
> giving
> > me
> > > > compression. Rest are equivalent to uncompressed storage,
> > > >
> > > > On Wed, May 12, 2021 at 11:16 AM nitin agarwal <
> nitingarg...@gmail.com
> > >
> > > > wrote:
> > > >
> > > >> You can read the data from the disk and see compression type.
> > > >> https://thehoard.blog/how-kafkas-storage-internals-work-3a29b02e026
> > > >>
> > > >> Thanks,
> > > >> Nitin
> > > >>
> > > >> On Wed, May 12, 2021 at 11:10 AM Shantanu Deshmukh <
> > > shantanu...@gmail.com
> > > >> >
> > > >> wrote:
> > > >>
> > > >> > I am trying snappy compression on my producer. Here's my setup
> > > >> >
> > > >> > Kafka - 2.0.0
> > > >> > Spring-Kafka - 2.1.2
> > > >> >
> > > >> > Here's my producer config
> > > >> >
> > > >> > compressed producer ==========
> > > >> >
> > > >> > configProps.put( ProducerConfig.BOOTSTRAP_SERVERS_CONFIG,
> > > >> >             bootstrapServer);
> > > >> >     configProps.put(
> > > >> >             ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
> > > >> >             StringSerializer.class);
> > > >> >     configProps.put(
> > > >> >             ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,
> > > >> >             StringSerializer.class);
> > > >> >     configProps.put(ProducerConfig.COMPRESSION_TYPE_CONFIG,
> > "snappy");
> > > >> >     configProps.put(ProducerConfig.LINGER_MS_CONFIG, 10);
> > > >> >
> > > >> > config of un-compressed producer ============
> > > >> >
> > > >> > configProps.put(
> > > >> >             ProducerConfig.BOOTSTRAP_SERVERS_CONFIG,
> > > >> >             bootstrapServer);
> > > >> >     configProps.put(
> > > >> >             ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
> > > >> >             StringSerializer.class);
> > > >> >     configProps.put(
> > > >> >             ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,
> > > >> >             StringSerializer.class);
> > > >> >
> > > >> > My payload is almost 1mb worth of string. After sending 1000
> > > compressed
> > > >> and
> > > >> > 1000 uncompressed such messages this is the result
> > > >> > =======================
> > > >> > [shantanu@oc0148610736 uncompressed-string-test-0]$ du -hsc
> > > >> > /data/compressed-string-test-0/*
> > > >> > 8.0K /data/compressed-string-test-0/00000000000000000000.index
> > > >> > 990M /data/compressed-string-test-0/00000000000000000000.log
> > > >> > 12K /data/compressed-string-test-0/00000000000000000000.timeindex
> > > >> > 4.0K /data/compressed-string-test-0/leader-epoch-checkpoint
> > > >> > 990M total
> > > >> >
> > > >> > [shantanu@oc0148610736 uncompressed-string-test-0]$ du -shc
> > > >> > /data/uncompressed-string-test-0/*
> > > >> > 8.0K
> /data/uncompressed-string-test-0/00000000000000000000.index
> > > >> > 992M    /data/uncompressed-string-test-0/00000000000000000000.log
> > > >> > 12K
> /data/uncompressed-string-test-0/00000000000000000000.timeindex
> > > >> > 4.0K    /data/uncompressed-string-test-0/leader-epoch-checkpoint
> > > >> > 992M    total
> > > >> > =======================
> > > >> >
> > > >> > Here we can see the difference is merely 2MB. Is compression even
> > > >> working?
> > > >> > I used dump-log-segment tool
> > > >> > =======================
> > > >> > [shantanu@oc0148610736 kafka_2.11-2.0.0]$ sh
> bin/kafka-run-class.sh
> > > >> > kafka.tools.DumpLogSegments --files
> > > >> > /data/compressed-string-test-0/00000000000000000000.log
> > > >> --print-data-log |
> > > >> > head | grep compresscodec
> > > >> >
> > > >> > offset: 0 position: 0 CreateTime: 1620744081357 isvalid: true
> > keysize:
> > > >> > -1 valuesize: 1039999 magic: 2 compresscodec: SNAPPY producerId:
> -1
> > > >> > producerEpoch: -1 sequence: -1 isTransactional: false headerKeys:
> []
> > > >> > payload:
> > > >> >
> > > >>
> > >
> >
> klxhbpyxmcazvhekqnltuenwhsewjjfmctcqyrppellyfqglfnvhqctlfplslhpuulknsncbgzzndizwmlnelotcbniyprdgihdazwn
> > > >> > =======================
> > > >> >
> > > >> > I can see SNAPPY is mentioned as compression codec. But the
> > difference
> > > >> > between compressed and uncompressed disk size is negligible.
> > > >> >
> > > >> > I tried gzip later on. And results are
> > > >> > =======================
> > > >> > [shantanu@oc0148610736 uncompressed-string-test-0]$ du -hsc
> > > >> > /data/compressed-string-test-0/*
> > > >> > 8.0K /data/compressed-string-test-0/00000000000000000000.index
> > > >> > 640M /data/compressed-string-test-0/00000000000000000000.log
> > > >> > 12K /data/compressed-string-test-0/00000000000000000000.timeindex
> > > >> > 4.0K /data/compressed-string-test-0/leader-epoch-checkpoint
> > > >> > 640M total
> > > >> > =======================
> > > >> >
> > > >> > So gzip seems to have worked somehow. I tried lz4 compression as
> > well.
> > > >> > Results were same as that of snappy.
> > > >> >
> > > >> > Is snappy/lz4 compression really working here? Gzip seems to be
> > > working
> > > >> but
> > > >> > I have read a lot that snappy gives best CPU usage to compression
> > > ratio
> > > >> > balance. So we want to go ahead with snappy.
> > > >> >
> > > >> > Please help
> > > >> >
> > > >> > *Thanks & Regards,*
> > > >> > *Shantanu*
> > > >> >
> > > >>
> > > >
> > >
> >
>

Reply via email to