Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Enables zstd (for real this time) #1574

Merged
merged 1 commit into from
Jan 23, 2020
Merged

Conversation

d1egoaz
Copy link
Contributor

@d1egoaz d1egoaz commented Jan 15, 2020

Zstd support was initially added to sarama before
https://issues.apache.org/jira/browse/KAFKA-4514
https://cwiki.apache.org/confluence/display/KAFKA/KIP-110%3A+Add+Codec+for+ZStandard+Compression
was done.

The final release added some changes like bumping the produce and fetch
requests to only allow new clients to use zstd.

This PR tries to do that, however, there are some other protocol changes
that are not addressed on this PR, and I'm not sure what would be the
effect of bumping the produce and fetch requests without filling the
protocol gaps.

Tries to solve #1252

TODO:

Update 2020-01-15 13:00 ET

  • it seems it's working fine after my latests commits 🎉

Now I need to figure it out what to do with the new producer response field, or just leave it like that?

Update 2020-01-15 11:00 ET

  • It still doesn't work :(~

It's however and improvement from the current state, this is the new error message:

producer/broker/1 state change to [closing] because kafka: error decoding packet: invalid length

current master result: (from #1252)

SARAMA: producer/broker/1 starting up
SARAMA: producer/broker/1 state change to [open] on sarama-test/0
SARAMA: Connected to broker at localhost:9092 (registered as #1)
SARAMA: producer/broker/1 state change to [closing] because EOF
SARAMA: Closed connection to broker localhost:9092
TEST:   sending message failed: EOF
SARAMA: Producer shutting down.
SARAMA: Closing Client
SARAMA: Closed connection to broker localhost:9092
SARAMA: producer/broker/1 shut down

Proposed change: (make testzstd)

╭─[10:44:05] diegoalvarez@d1egoaz-MBP/ ~/src/github.com/Shopify/sarama
( diego_zstd-support-kafka-2_1_0_0: ✔
╰─✘ make testzstd
go test -run TestSaramaZSTD
>>> kafka version 2.1.0
Initializing new client
client/metadata fetching metadata for all topics from broker localhost:9092
Connected to broker at localhost:9092 (unregistered)
>>> encoder_decoder.go on versionedDecode, realDecoder, version: 5
>>> encoder_decoder.go helper.off != len(buf), off: 163, len: 163, version: 5
client/brokers registered new broker #1 at localhost:9092
Successfully initialized new client
producer/broker/1 starting up
producer/broker/1 state change to [open] on sarama-test/0
>>> produce_set.go buildRequest() set req.Version = 7 with compression zstd
Connected to broker at localhost:9092 (registered as #1)
>>> produce_request.go requiredVersion r.Version 7
>>> record_batch.go on encode
>>> record_batch.go/encodeRecords() compressing message on encodeRecords
>>> compressing with zstd
>>> record_batch.go/encodeRecords() NO err compressing message on encodeRecords
>>> record_batch.go put raw bytes on encode
>>> record_batch.go pe.pop on encode
>>> record_batch.go NO err pe.pop on encode
>>> record_batch.go on encode
>>> record_batch.go put raw bytes on encode
>>> record_batch.go pe.pop on encode
>>> record_batch.go NO err pe.pop on encode
>>> encoder_decoder.go on versionedDecode, realDecoder, version: 7
>>> encoder_decoder.go helper.off != len(buf), off: 47, len: 55, version: 7
>>> encoder_decoder.go on versionedDecode, version 7
producer/broker/1 state change to [closing] because kafka: error decoding packet: invalid length
Closed connection to broker localhost:9092
Producer shutting down.
Closing Client
Closed connection to broker localhost:9092
producer/broker/1 input chan closed
producer/broker/1 shut down
--- FAIL: TestSaramaZSTD (0.03s)
    zstd_test.go:45: TEST:   sending message failed: kafka: error decoding packet: invalid length
FAIL
exit status 1
FAIL    github.com/Shopify/sarama       5.422s

Code is failing here:

>>> encoder_decoder.go helper.off != len(buf), off: 47, len: 55, version: 7

I'm not sure what I'm missing, I'm currently 👀 the java source code.

Any help would be appreciated.

@d1egoaz
Copy link
Contributor Author

d1egoaz commented Jan 15, 2020

ping @bai @bobrik 🙏

@d1egoaz
Copy link
Contributor Author

d1egoaz commented Jan 15, 2020

a little bit of more info, I think the message got produced, the error is on handling the produce response.
I'm using kafkacat to consume from the topic, and I'm seeing messages being added to the topic.

I'm also running kafkacat with debug mode, and I think it's consuming messages from a topic with zstd compression according to on sarama-test [0] fetch queue (qlen 37, v2, last_offset 36, 0 ctrl msgs, zstd)

kafkacat -Cb localhost:9092 -t sarama-test -d all

%7|1579106515.545|FETCH|rdkafka#consumer-1| [thrd:localhost:9092/bootstrap]: localhost:9092/1: Topic sarama-test [0] MessageSet size 3441, error "Success", MaxOffset 37, LSO 37, Ver 2/2
%7|1579106515.545|CONSUME|rdkafka#consumer-1| [thrd:localhost:9092/bootstrap]: localhost:9092/1: Enqueue 37 message(s) (444 bytes, 37 ops) on sarama-test [0] fetch queue (qlen 37, v2, last_offset 36, 0 ctrl msgs, zstd)
%7|1579106515.545|FETCH|rdkafka#consumer-1| [thrd:localhost:9092/bootstrap]: localhost:9092/1: Fetch topic sarama-test [0] at offset 37 (v2)
%7|1579106515.545|FETCH|rdkafka#consumer-1| [thrd:localhost:9092/bootstrap]: localhost:9092/1: Fetch 1/1/1 toppar(s)
%7|1579106515.545|SEND|rdkafka#consumer-1| [thrd:localhost:9092/bootstrap]: localhost:9092/1: Sent FetchRequest (v4, 75 bytes @ 0, CorrId 5)
hello world!
hello world!
hello world!
hello world!

@d1egoaz
Copy link
Contributor Author

d1egoaz commented Jan 15, 2020

Error is being fired by:

https://github.com/Shopify/sarama/blob/diego_zstd-support-kafka-2_1_0_0/broker.go#L735-L751

Wich confirms my data that the message is actually being sent.

@d1egoaz
Copy link
Contributor Author

d1egoaz commented Jan 15, 2020

It looks like we haven't read all bytes from the response, according to the error message and https://github.com/Shopify/sarama/blob/diego_zstd-support-kafka-2_1_0_0/real_decoder.go
helper.off != len(buf), off: 47, len: 55
So, it's expecting that we should read another 8 bytes, wondering if that's coming from a protocol change (the gaps I mentioned above)

@d1egoaz
Copy link
Contributor Author

d1egoaz commented Jan 15, 2020

found the missing bytes! 🎉

Will send a commit soon, as I need to add some tests

@d1egoaz
Copy link
Contributor Author

d1egoaz commented Jan 15, 2020

I noticed you were interested on this @lizthegrey might want help me to test it and report back, thanks

@d1egoaz d1egoaz marked this pull request as ready for review January 15, 2020 18:05
@d1egoaz d1egoaz requested review from bai and varun06 January 15, 2020 18:06
@d1egoaz d1egoaz changed the title Enables zstd bumping the required things (not working yet) Enables zstd (for real this time) Jan 15, 2020
record_batch.go Outdated Show resolved Hide resolved
record_batch.go Outdated Show resolved Hide resolved
@lizthegrey
Copy link
Contributor

I noticed you were interested on this @lizthegrey might want help me to test it and report back, thanks

Oh thank fuck. Yes, we'd be delighted to test this. Let me figure out how to feature flag this... cc @ianwilkes @tredman

Copy link
Contributor

@bai bai left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Apart from nits mentioned, code LGTM. Good stuff, thanks.

@d1egoaz
Copy link
Contributor Author

d1egoaz commented Jan 15, 2020

it looks like consumer is still not working: EOF

2020/01/15 15:52:51 error: &sarama.ConsumerError{Topic:"my-zstd-topic", Partition:0, Err:(*errors.errorString)(0xc00006e040)}
2020/01/15 15:52:51 error.err: &errors.errorString{s:"EOF"}

might be similar to the producer part, protocol changes between latest sarama protocol support and request.Version = 10

@d1egoaz
Copy link
Contributor Author

d1egoaz commented Jan 15, 2020

not looking good for the consumer side :(
We need to investigate what of these fields needs to be added to the fetch request/response
https://github.com/apache/kafka/blob/99a4068c5ca61951d70b9e647ead3b08a2af4309/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java#L120-L197

It's what @lizthegrey mentioned here #1443

@d1egoaz
Copy link
Contributor Author

d1egoaz commented Jan 16, 2020

Wondering if we should merge this as it's right now and leave a note like:
image

looks like the consumer part has more protocol gaps that needs to be fixed, I can try to work on that on the next weeks in a new PR.

What do you think @bai , we also need to wait a little bit for more confirmation that producing works.

I'm still not sure how ruby-kafka added zstd support with only this minor changes https://github.com/zendesk/ruby-kafka/pull/724/files

I'd like to test it to see if that even works 🤷‍♂

@d1egoaz d1egoaz changed the title Enables zstd (for real this time) Enables zstd (for real this time) - only works for producing messages Jan 16, 2020
@dnwe
Copy link
Collaborator

dnwe commented Jan 16, 2020

@d1egoaz can you add this patch to your branch (git am -i file.patch) and see if it fixes your consumer issues? I added the missing protocol encode/decode all the way up to the required version 10 for FetchRequest + FetchResponse. With this patch I seemed to be able to roundtrip zstd compressed messages fine.

From aea82107ffee2880ed8394326da7efa14ef586d0 Mon Sep 17 00:00:00 2001
From: Dominic Evans <dominic.evans@uk.ibm.com>
Date: Thu, 16 Jan 2020 23:23:36 +0000
Subject: [PATCH] fix: fill in the Fetch{Request,Response} protocol

In order to consume zstd-compressed records the consumer needs to send
and receive version 10 FetchRequest/FetchResponses, but they need to do
so in a well-formed manner that adheres to the encoding format.

Ref: https://kafka.apache.org/protocol

Signed-off-by: Dominic Evans <dominic.evans@uk.ibm.com>
---
 fetch_request.go      | 123 +++++++++++++++++++++++++++++++++++++-----
 fetch_request_test.go |  44 +++++++++------
 fetch_response.go     |  33 +++++++++++-
 request.go            |   2 +-
 4 files changed, 172 insertions(+), 30 deletions(-)

diff --git a/fetch_request.go b/fetch_request.go
index f2a4643..d5f0776 100644
--- a/fetch_request.go
+++ b/fetch_request.go
@@ -1,20 +1,41 @@
 package sarama
 
 type fetchRequestBlock struct {
-	fetchOffset int64
-	maxBytes    int32
+	Version            int16
+	currentLeaderEpoch int32
+	fetchOffset        int64
+	logStartOffset     int64
+	maxBytes           int32
 }
 
-func (b *fetchRequestBlock) encode(pe packetEncoder) error {
+func (b *fetchRequestBlock) encode(pe packetEncoder, version int16) error {
+	b.Version = version
+	if b.Version >= 9 {
+		pe.putInt32(b.currentLeaderEpoch)
+	}
 	pe.putInt64(b.fetchOffset)
+	if b.Version >= 5 {
+		pe.putInt64(b.logStartOffset)
+	}
 	pe.putInt32(b.maxBytes)
 	return nil
 }
 
-func (b *fetchRequestBlock) decode(pd packetDecoder) (err error) {
+func (b *fetchRequestBlock) decode(pd packetDecoder, version int16) (err error) {
+	b.Version = version
+	if b.Version >= 9 {
+		if b.currentLeaderEpoch, err = pd.getInt32(); err != nil {
+			return err
+		}
+	}
 	if b.fetchOffset, err = pd.getInt64(); err != nil {
 		return err
 	}
+	if b.Version >= 5 {
+		if b.logStartOffset, err = pd.getInt64(); err != nil {
+			return err
+		}
+	}
 	if b.maxBytes, err = pd.getInt32(); err != nil {
 		return err
 	}
@@ -25,12 +46,15 @@ func (b *fetchRequestBlock) decode(pd packetDecoder) (err error) {
 // https://issues.apache.org/jira/browse/KAFKA-2063 for a discussion of the issues leading up to that.  The KIP is at
 // https://cwiki.apache.org/confluence/display/KAFKA/KIP-74%3A+Add+Fetch+Response+Size+Limit+in+Bytes
 type FetchRequest struct {
-	MaxWaitTime int32
-	MinBytes    int32
-	MaxBytes    int32
-	Version     int16
-	Isolation   IsolationLevel
-	blocks      map[string]map[int32]*fetchRequestBlock
+	MaxWaitTime  int32
+	MinBytes     int32
+	MaxBytes     int32
+	Version      int16
+	Isolation    IsolationLevel
+	SessionID    int32
+	SessionEpoch int32
+	blocks       map[string]map[int32]*fetchRequestBlock
+	forgotten    map[string][]int32
 }
 
 type IsolationLevel int8
@@ -50,6 +74,10 @@ func (r *FetchRequest) encode(pe packetEncoder) (err error) {
 	if r.Version >= 4 {
 		pe.putInt8(int8(r.Isolation))
 	}
+	if r.Version >= 7 {
+		pe.putInt32(r.SessionID)
+		pe.putInt32(r.SessionEpoch)
+	}
 	err = pe.putArrayLength(len(r.blocks))
 	if err != nil {
 		return err
@@ -65,17 +93,38 @@ func (r *FetchRequest) encode(pe packetEncoder) (err error) {
 		}
 		for partition, block := range blocks {
 			pe.putInt32(partition)
-			err = block.encode(pe)
+			err = block.encode(pe, r.Version)
 			if err != nil {
 				return err
 			}
 		}
 	}
+	if r.Version >= 7 {
+		err = pe.putArrayLength(len(r.forgotten))
+		if err != nil {
+			return err
+		}
+		for topic, partitions := range r.forgotten {
+			err = pe.putString(topic)
+			if err != nil {
+				return err
+			}
+			err = pe.putArrayLength(len(partitions))
+			if err != nil {
+				return err
+			}
+			for _, partition := range partitions {
+				pe.putInt32(partition)
+			}
+		}
+	}
+
 	return nil
 }
 
 func (r *FetchRequest) decode(pd packetDecoder, version int16) (err error) {
 	r.Version = version
+
 	if _, err = pd.getInt32(); err != nil {
 		return err
 	}
@@ -97,6 +146,16 @@ func (r *FetchRequest) decode(pd packetDecoder, version int16) (err error) {
 		}
 		r.Isolation = IsolationLevel(isolation)
 	}
+	if r.Version >= 7 {
+		r.SessionID, err = pd.getInt32()
+		if err != nil {
+			return err
+		}
+		r.SessionEpoch, err = pd.getInt32()
+		if err != nil {
+			return err
+		}
+	}
 	topicCount, err := pd.getArrayLength()
 	if err != nil {
 		return err
@@ -121,12 +180,43 @@ func (r *FetchRequest) decode(pd packetDecoder, version int16) (err error) {
 				return err
 			}
 			fetchBlock := &fetchRequestBlock{}
-			if err = fetchBlock.decode(pd); err != nil {
+			if err = fetchBlock.decode(pd, r.Version); err != nil {
 				return err
 			}
 			r.blocks[topic][partition] = fetchBlock
 		}
 	}
+
+	if r.Version >= 7 {
+		forgottenCount, err := pd.getArrayLength()
+		if err != nil {
+			return err
+		}
+		if forgottenCount == 0 {
+			return nil
+		}
+		r.forgotten = make(map[string][]int32)
+		for i := 0; i < forgottenCount; i++ {
+			topic, err := pd.getString()
+			if err != nil {
+				return err
+			}
+			partitionCount, err := pd.getArrayLength()
+			if err != nil {
+				return err
+			}
+			r.forgotten[topic] = make([]int32, partitionCount)
+
+			for j := 0; j < partitionCount; j++ {
+				partition, err := pd.getInt32()
+				if err != nil {
+					return err
+				}
+				r.forgotten[topic][j] = partition
+			}
+		}
+	}
+
 	return nil
 }
 
@@ -138,6 +228,7 @@ func (r *FetchRequest) version() int16 {
 	return r.Version
 }
 
+// FIXME: get these correct
 func (r *FetchRequest) requiredVersion() KafkaVersion {
 	switch r.Version {
 	case 1:
@@ -160,13 +251,21 @@ func (r *FetchRequest) AddBlock(topic string, partitionID int32, fetchOffset int
 		r.blocks = make(map[string]map[int32]*fetchRequestBlock)
 	}
 
+	if r.Version >= 7 && r.forgotten == nil {
+		r.forgotten = make(map[string][]int32)
+	}
+
 	if r.blocks[topic] == nil {
 		r.blocks[topic] = make(map[int32]*fetchRequestBlock)
 	}
 
 	tmp := new(fetchRequestBlock)
+	tmp.Version = r.Version
 	tmp.maxBytes = maxBytes
 	tmp.fetchOffset = fetchOffset
+	if r.Version >= 9 {
+		tmp.currentLeaderEpoch = int32(-1)
+	}
 
 	r.blocks[topic][partitionID] = tmp
 }
diff --git a/fetch_request_test.go b/fetch_request_test.go
index 1a94c2d..2fdd905 100644
--- a/fetch_request_test.go
+++ b/fetch_request_test.go
@@ -29,20 +29,32 @@ var (
 )
 
 func TestFetchRequest(t *testing.T) {
-	request := new(FetchRequest)
-	testRequest(t, "no blocks", request, fetchRequestNoBlocks)
-
-	request.MaxWaitTime = 0x20
-	request.MinBytes = 0xEF
-	testRequest(t, "with properties", request, fetchRequestWithProperties)
-
-	request.MaxWaitTime = 0
-	request.MinBytes = 0
-	request.AddBlock("topic", 0x12, 0x34, 0x56)
-	testRequest(t, "one block", request, fetchRequestOneBlock)
-
-	request.Version = 4
-	request.MaxBytes = 0xFF
-	request.Isolation = ReadCommitted
-	testRequest(t, "one block v4", request, fetchRequestOneBlockV4)
+	t.Run("no blocks", func(t *testing.T) {
+		request := new(FetchRequest)
+		testRequest(t, "no blocks", request, fetchRequestNoBlocks)
+	})
+
+	t.Run("with properties", func(t *testing.T) {
+		request := new(FetchRequest)
+		request.MaxWaitTime = 0x20
+		request.MinBytes = 0xEF
+		testRequest(t, "with properties", request, fetchRequestWithProperties)
+	})
+
+	t.Run("one block", func(t *testing.T) {
+		request := new(FetchRequest)
+		request.MaxWaitTime = 0
+		request.MinBytes = 0
+		request.AddBlock("topic", 0x12, 0x34, 0x56)
+		testRequest(t, "one block", request, fetchRequestOneBlock)
+	})
+
+	t.Run("one block v4", func(t *testing.T) {
+		request := new(FetchRequest)
+		request.Version = 4
+		request.MaxBytes = 0xFF
+		request.Isolation = ReadCommitted
+		request.AddBlock("topic", 0x12, 0x34, 0x56)
+		testRequest(t, "one block v4", request, fetchRequestOneBlockV4)
+	})
 }
diff --git a/fetch_response.go b/fetch_response.go
index 3afc187..79cc015 100644
--- a/fetch_response.go
+++ b/fetch_response.go
@@ -33,6 +33,7 @@ type FetchResponseBlock struct {
 	Err                 KError
 	HighWaterMarkOffset int64
 	LastStableOffset    int64
+	LogStartOffset      int64
 	AbortedTransactions []*AbortedTransaction
 	Records             *Records // deprecated: use FetchResponseBlock.RecordsSet
 	RecordsSet          []*Records
@@ -57,6 +58,13 @@ func (b *FetchResponseBlock) decode(pd packetDecoder, version int16) (err error)
 			return err
 		}
 
+		if version >= 5 {
+			b.LogStartOffset, err = pd.getInt64()
+			if err != nil {
+				return err
+			}
+		}
+
 		numTransact, err := pd.getArrayLength()
 		if err != nil {
 			return err
@@ -166,6 +174,10 @@ func (b *FetchResponseBlock) encode(pe packetEncoder, version int16) (err error)
 	if version >= 4 {
 		pe.putInt64(b.LastStableOffset)
 
+		if version >= 5 {
+			pe.putInt64(b.LogStartOffset)
+		}
+
 		if err = pe.putArrayLength(len(b.AbortedTransactions)); err != nil {
 			return err
 		}
@@ -200,7 +212,9 @@ func (b *FetchResponseBlock) getAbortedTransactions() []*AbortedTransaction {
 type FetchResponse struct {
 	Blocks        map[string]map[int32]*FetchResponseBlock
 	ThrottleTime  time.Duration
-	Version       int16 // v1 requires 0.9+, v2 requires 0.10+
+	ErrorCode     int16
+	SessionID     int32
+	Version       int16
 	LogAppendTime bool
 	Timestamp     time.Time
 }
@@ -216,6 +230,17 @@ func (r *FetchResponse) decode(pd packetDecoder, version int16) (err error) {
 		r.ThrottleTime = time.Duration(throttle) * time.Millisecond
 	}
 
+	if r.Version >= 7 {
+		r.ErrorCode, err = pd.getInt16()
+		if err != nil {
+			return err
+		}
+		r.SessionID, err = pd.getInt32()
+		if err != nil {
+			return err
+		}
+	}
+
 	numTopics, err := pd.getArrayLength()
 	if err != nil {
 		return err
@@ -258,6 +283,11 @@ func (r *FetchResponse) encode(pe packetEncoder) (err error) {
 		pe.putInt32(int32(r.ThrottleTime / time.Millisecond))
 	}
 
+	if r.Version >= 7 {
+		pe.putInt16(r.ErrorCode)
+		pe.putInt32(r.SessionID)
+	}
+
 	err = pe.putArrayLength(len(r.Blocks))
 	if err != nil {
 		return err
@@ -294,6 +324,7 @@ func (r *FetchResponse) version() int16 {
 	return r.Version
 }
 
+// FIXME: get these correct
 func (r *FetchResponse) requiredVersion() KafkaVersion {
 	switch r.Version {
 	case 1:
diff --git a/request.go b/request.go
index 97437d6..6e4ad87 100644
--- a/request.go
+++ b/request.go
@@ -105,7 +105,7 @@ func allocateBody(key, version int16) protocolBody {
 	case 0:
 		return &ProduceRequest{}
 	case 1:
-		return &FetchRequest{}
+		return &FetchRequest{Version: version}
 	case 2:
 		return &OffsetRequest{Version: version}
 	case 3:
-- 
2.24.0

@dnwe
Copy link
Collaborator

dnwe commented Jan 17, 2020

I've pushed the above patch up under a separate PR here #1582

@d1egoaz
Copy link
Contributor Author

d1egoaz commented Jan 17, 2020

Great @dnwe I'll try to give it a go, I'm currently checking the producer as I've got some other error decoding packet: invalid length messages in some local tests, going to see what other protocol gaps are also needed for the producer

@dnwe
Copy link
Collaborator

dnwe commented Jan 17, 2020

@d1egoaz hmm I had already checked the ProduceRequest and it seemed there weren't any new fields required. V4 added a new error type, V5 and V6 were identical to V4 and V7 just meant "supports zstd"

@d1egoaz
Copy link
Contributor Author

d1egoaz commented Jan 17, 2020

@dnwe see my latest commit, I added the new field in the wrong section 🤦‍♂
Now it's working fine! 🎉
7e314f9

@d1egoaz
Copy link
Contributor Author

d1egoaz commented Jan 17, 2020

@lizthegrey my tests found and error that was fixed, on my local env and some extended tests this has been working great!

@dnwe also added support for the consumer and it worked great for me and for CI.

Please let us know if you/your team were able to test this
thanks

@d1egoaz

This comment has been minimized.

@d1egoaz d1egoaz changed the title Enables zstd (for real this time) - only works for producing messages Enables zstd (for real this time) Jan 17, 2020
@d1egoaz d1egoaz force-pushed the diego_zstd-support-kafka-2_1_0_0 branch 2 times, most recently from c394c83 to eca4c40 Compare January 21, 2020 16:44
consumer.go Show resolved Hide resolved
Zstd support was initially added to sarama before
https://issues.apache.org/jira/browse/KAFKA-4514
https://cwiki.apache.org/confluence/display/KAFKA/KIP-110%3A+Add+Codec+for+ZStandard+Compression
was done.

The final release added some changes like bumping the produce and fetch
requests to only allow new clients to use zstd.

This PR tries to do that, however, there are some other protocol changes
that are not addressed on this PR, and I'm not sure what would be the
effect of bumping the produce and fetch requests without filling the
protocol gaps.
@d1egoaz d1egoaz force-pushed the diego_zstd-support-kafka-2_1_0_0 branch from eca4c40 to 37faed7 Compare January 21, 2020 18:48
@d1egoaz d1egoaz merged commit 30e7094 into master Jan 23, 2020
@d1egoaz d1egoaz deleted the diego_zstd-support-kafka-2_1_0_0 branch January 23, 2020 21:28
@lizthegrey
Copy link
Contributor

Thank you for saving us 35% off our Kafka bill <3 <3
Screenshot 2020-01-31 at 2 51 42 PM

@varun06
Copy link
Contributor

varun06 commented Jan 31, 2020 via email

@lizthegrey
Copy link
Contributor

Screenshot 2020-02-03 at 11 18 28 AM
Here are our results after the weekend...

@varun06
Copy link
Contributor

varun06 commented Feb 3, 2020

Love that, thanks everyone for making it happen.

@himanshpal
Copy link

We are seeing high memory usage breaching container hard-memory limits & resulting in OOM after moving to Zstd from Snappy. Is this expected ?

Screenshot 2020-07-16 at 3 21 32 PM

@d1egoaz
Copy link
Contributor Author

d1egoaz commented Jul 18, 2020

We are seeing high memory usage breaching container hard-memory limits & resulting in OOM after moving to Zstd from Snappy. Is this expected ?

Screenshot 2020-07-16 at 3 21 32 PM

Might want also to report this in https://github.com/klauspost/compress/

@funny-falcon-at-joomcode

Couldn't it be related? klauspost/compress#264

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants