-
Notifications
You must be signed in to change notification settings - Fork 1.8k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Merge pull request #588 from dim/kafka-09-consumer-groups
Consumer groups on Kafka 0.9
- Loading branch information
Showing
8 changed files
with
301 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,94 @@ | ||
package sarama | ||
|
||
type ConsumerGroupMemberMetadata struct { | ||
Version int16 | ||
Topics []string | ||
UserData []byte | ||
} | ||
|
||
func (m *ConsumerGroupMemberMetadata) encode(pe packetEncoder) error { | ||
pe.putInt16(m.Version) | ||
|
||
if err := pe.putStringArray(m.Topics); err != nil { | ||
return err | ||
} | ||
|
||
if err := pe.putBytes(m.UserData); err != nil { | ||
return err | ||
} | ||
|
||
return nil | ||
} | ||
|
||
func (m *ConsumerGroupMemberMetadata) decode(pd packetDecoder) (err error) { | ||
if m.Version, err = pd.getInt16(); err != nil { | ||
return | ||
} | ||
|
||
if m.Topics, err = pd.getStringArray(); err != nil { | ||
return | ||
} | ||
|
||
if m.UserData, err = pd.getBytes(); err != nil { | ||
return | ||
} | ||
|
||
return nil | ||
} | ||
|
||
type ConsumerGroupMemberAssignment struct { | ||
Version int16 | ||
Topics map[string][]int32 | ||
UserData []byte | ||
} | ||
|
||
func (m *ConsumerGroupMemberAssignment) encode(pe packetEncoder) error { | ||
pe.putInt16(m.Version) | ||
|
||
if err := pe.putArrayLength(len(m.Topics)); err != nil { | ||
return err | ||
} | ||
|
||
for topic, partitions := range m.Topics { | ||
if err := pe.putString(topic); err != nil { | ||
return err | ||
} | ||
if err := pe.putInt32Array(partitions); err != nil { | ||
return err | ||
} | ||
} | ||
|
||
if err := pe.putBytes(m.UserData); err != nil { | ||
return err | ||
} | ||
|
||
return nil | ||
} | ||
|
||
func (m *ConsumerGroupMemberAssignment) decode(pd packetDecoder) (err error) { | ||
if m.Version, err = pd.getInt16(); err != nil { | ||
return | ||
} | ||
|
||
var topicLen int | ||
if topicLen, err = pd.getArrayLength(); err != nil { | ||
return | ||
} | ||
|
||
m.Topics = make(map[string][]int32, topicLen) | ||
for i := 0; i < topicLen; i++ { | ||
var topic string | ||
if topic, err = pd.getString(); err != nil { | ||
return | ||
} | ||
if m.Topics[topic], err = pd.getInt32Array(); err != nil { | ||
return | ||
} | ||
} | ||
|
||
if m.UserData, err = pd.getBytes(); err != nil { | ||
return | ||
} | ||
|
||
return nil | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,77 @@ | ||
package sarama | ||
|
||
import ( | ||
"bytes" | ||
"reflect" | ||
"testing" | ||
) | ||
|
||
var ( | ||
groupMemberMetadata = []byte{ | ||
0, 1, // Version | ||
0, 0, 0, 2, // Topic array length | ||
0, 3, 'o', 'n', 'e', // Topic one | ||
0, 3, 't', 'w', 'o', // Topic two | ||
0, 0, 0, 3, 0x01, 0x02, 0x03, // Userdata | ||
} | ||
groupMemberAssignment = []byte{ | ||
0, 1, // Version | ||
0, 0, 0, 2, // Topic array length | ||
0, 3, 'o', 'n', 'e', // Topic one | ||
0, 0, 0, 3, // Topic one, partition array length | ||
0, 0, 0, 0, 0, 0, 0, 2, 0, 0, 0, 4, // 0, 2, 4 | ||
0, 3, 't', 'w', 'o', // Topic two | ||
0, 0, 0, 2, // Topic two, partition array length | ||
0, 0, 0, 1, 0, 0, 0, 3, // 1, 3 | ||
0, 0, 0, 3, 0x01, 0x02, 0x03, // Userdata | ||
} | ||
) | ||
|
||
func TestConsumerGroupMemberMetadata(t *testing.T) { | ||
meta := &ConsumerGroupMemberMetadata{ | ||
Version: 1, | ||
Topics: []string{"one", "two"}, | ||
UserData: []byte{0x01, 0x02, 0x03}, | ||
} | ||
|
||
buf, err := encode(meta) | ||
if err != nil { | ||
t.Error("Failed to encode data", err) | ||
} else if !bytes.Equal(groupMemberMetadata, buf) { | ||
t.Errorf("Encoded data does not match expectation\nexpected: %v\nactual: %v", groupMemberMetadata, buf) | ||
} | ||
|
||
meta2 := new(ConsumerGroupMemberMetadata) | ||
err = decode(buf, meta2) | ||
if err != nil { | ||
t.Error("Failed to decode data", err) | ||
} else if !reflect.DeepEqual(meta, meta2) { | ||
t.Errorf("Encoded data does not match expectation\nexpected: %v\nactual: %v", meta, meta2) | ||
} | ||
} | ||
|
||
func TestConsumerGroupMemberAssignment(t *testing.T) { | ||
amt := &ConsumerGroupMemberAssignment{ | ||
Version: 1, | ||
Topics: map[string][]int32{ | ||
"one": []int32{0, 2, 4}, | ||
"two": []int32{1, 3}, | ||
}, | ||
UserData: []byte{0x01, 0x02, 0x03}, | ||
} | ||
|
||
buf, err := encode(amt) | ||
if err != nil { | ||
t.Error("Failed to encode data", err) | ||
} else if !bytes.Equal(groupMemberAssignment, buf) { | ||
t.Errorf("Encoded data does not match expectation\nexpected: %v\nactual: %v", groupMemberAssignment, buf) | ||
} | ||
|
||
amt2 := new(ConsumerGroupMemberAssignment) | ||
err = decode(buf, amt2) | ||
if err != nil { | ||
t.Error("Failed to decode data", err) | ||
} else if !reflect.DeepEqual(amt, amt2) { | ||
t.Errorf("Encoded data does not match expectation\nexpected: %v\nactual: %v", amt, amt2) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters