Skip to content

feat: Kafka 4.0 support #1384

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

Draft
wants to merge 12 commits into
base: main
Choose a base branch
from
55 changes: 54 additions & 1 deletion .circleci/config.yml
Original file line number Diff line number Diff line change
Expand Up @@ -178,6 +178,58 @@ jobs:
entrypoint: *entrypoint
steps: *steps

# NOTE: this fails quite often due to Java heap errors from Kafka.
# Once we switch to Github actions and can use larger instances, we can
# set the heap size and enable these
# kafka-400:
# working_directory: *working_directory
# environment:
# KAFKA_VERSION: "4.0.0"

# # Need to skip nettest to avoid these kinds of errors:
# # --- FAIL: TestConn/nettest (17.56s)
# # --- FAIL: TestConn/nettest/PingPong (7.40s)
# # conntest.go:112: unexpected Read error: [7] Request Timed Out: the request exceeded the user-specified time limit in the request
# # conntest.go:118: mismatching value: got 77, want 78
# # conntest.go:118: mismatching value: got 78, want 79
# # ...
# #
# # TODO: Figure out why these are happening and fix them (they don't appear to be new).
# KAFKA_SKIP_NETTEST: "1"
# docker:
# - image: circleci/golang
# - image: bitnami/kafka:4.0.0
# ports:
# - 9092:9092
# - 9093:9093
# environment:
# KAFKA_CFG_NODE_ID: 1
# KAFKA_CFG_BROKER_ID: 1
# KAFKA_CFG_PROCESS_ROLES: broker,controller
# KAFKA_CFG_ADVERTISED_HOST_NAME: 'localhost'
# KAFKA_CFG_CONTROLLER_LISTENER_NAMES: CONTROLLER
# KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: CONTROLLER:PLAINTEXT,PLAIN:PLAINTEXT,SASL:SASL_PLAINTEXT
# KAFKA_CFG_LISTENERS: CONTROLLER://:9094,PLAIN://:9092,SASL://:9093
# KAFKA_CFG_ADVERTISED_LISTENERS: PLAIN://localhost:9092,SASL://localhost:9093
# KAFKA_CFG_INTER_BROKER_LISTENER_NAME: PLAIN
# KAFKA_CFG_SASL_ENABLED_MECHANISMS: 'PLAIN,SCRAM-SHA-256,SCRAM-SHA-512'
# KAFKA_CFG_CONTROLLER_QUORUM_VOTERS: 1@localhost:9094
# ALLOW_PLAINTEXT_LISTENER: yes
# KAFKA_CFG_ALLOW_EVERYONE_IF_NO_ACL_FOUND: 'true'
# KAFKA_OPTS: "-Djava.security.auth.login.config=/opt/bitnami/kafka/config/kafka_jaas.conf"
# KAFKA_CFG_AUTO_CREATE_TOPICS_ENABLE: 'true'
# KAFKA_CFG_DELETE_TOPIC_ENABLE: 'true'
# KAFKA_CFG_MESSAGE_MAX_BYTES: '200000000'
# KAFKA_CFG_AUTHORIZER_CLASS_NAME: 'org.apache.kafka.metadata.authorizer.StandardAuthorizer'
# KAFKA_CFG_SUPER_USERS: User:adminscram256;User:adminscram512;User:adminplain
# KAFKA_CLIENT_USERS: adminscram256,adminscram512,adminplain
# KAFKA_CLIENT_PASSWORDS: admin-secret-256,admin-secret-512,admin-secret
# KAFKA_CLIENT_SASL_MECHANISMS: SCRAM-SHA-256,SCRAM-SHA-512,PLAIN
# KAFKA_INTER_BROKER_USER: adminscram512
# KAFKA_INTER_BROKER_PASSWORD: admin-secret-512
# KAFKA_CFG_SASL_MECHANISM_INTER_BROKER_PROTOCOL: SCRAM-SHA-512
# steps: *steps

workflows:
version: 2
run:
Expand All @@ -186,4 +238,5 @@ workflows:
- kafka-010
- kafka-270
- kafka-281
- kafka-370
- kafka-370
#- kafka-400
3 changes: 3 additions & 0 deletions alterpartitionreassignments_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@ package kafka
import (
"context"
"testing"
"time"

ktesting "github.com/segmentio/kafka-go/testing"
)
Expand Down Expand Up @@ -35,6 +36,7 @@ func TestClientAlterPartitionReassignments(t *testing.T) {
BrokerIDs: []int{1},
},
},
Timeout: 5 * time.Second,
},
)

Expand Down Expand Up @@ -96,6 +98,7 @@ func TestClientAlterPartitionReassignmentsMultiTopics(t *testing.T) {
BrokerIDs: []int{1},
},
},
Timeout: 5 * time.Second,
},
)

Expand Down
19 changes: 12 additions & 7 deletions conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -306,7 +306,7 @@ func (c *Conn) Brokers() ([]Broker, error) {

// DeleteTopics deletes the specified topics.
func (c *Conn) DeleteTopics(topics ...string) error {
_, err := c.deleteTopics(deleteTopicsRequestV0{
_, err := c.deleteTopics(deleteTopicsRequest{
Topics: topics,
})
return err
Expand Down Expand Up @@ -368,12 +368,17 @@ func (c *Conn) heartbeat(request heartbeatRequestV0) (heartbeatResponseV0, error
// joinGroup attempts to join a consumer group
//
// See http://kafka.apache.org/protocol.html#The_Messages_JoinGroup
func (c *Conn) joinGroup(request joinGroupRequestV1) (joinGroupResponseV1, error) {
var response joinGroupResponseV1
func (c *Conn) joinGroup(request joinGroupRequest) (joinGroupResponse, error) {
version, err := c.negotiateVersion(joinGroup, v1, v2)
if err != nil {
return joinGroupResponse{}, err
}

err := c.writeOperation(
response := joinGroupResponse{v: version}

err = c.writeOperation(
func(deadline time.Time, id int32) error {
return c.writeRequest(joinGroup, v1, id, request)
return c.writeRequest(joinGroup, version, id, request)
},
func(deadline time.Time, size int) error {
return expectZeroSize(func() (remain int, err error) {
Expand All @@ -382,10 +387,10 @@ func (c *Conn) joinGroup(request joinGroupRequestV1) (joinGroupResponseV1, error
},
)
if err != nil {
return joinGroupResponseV1{}, err
return joinGroupResponse{}, err
}
if response.ErrorCode != 0 {
return joinGroupResponseV1{}, Error(response.ErrorCode)
return joinGroupResponse{}, Error(response.ErrorCode)
}

return response, nil
Expand Down
13 changes: 7 additions & 6 deletions conn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,8 +13,9 @@ import (
"testing"
"time"

ktesting "github.com/segmentio/kafka-go/testing"
"golang.org/x/net/nettest"

ktesting "github.com/segmentio/kafka-go/testing"
)

type timeout struct{}
Expand Down Expand Up @@ -679,10 +680,10 @@ func waitForCoordinator(t *testing.T, conn *Conn, groupID string) {
func createGroup(t *testing.T, conn *Conn, groupID string) (generationID int32, memberID string, stop func()) {
waitForCoordinator(t, conn, groupID)

join := func() (joinGroup joinGroupResponseV1) {
join := func() (joinGroup joinGroupResponse) {
var err error
for attempt := 0; attempt < 10; attempt++ {
joinGroup, err = conn.joinGroup(joinGroupRequestV1{
joinGroup, err = conn.joinGroup(joinGroupRequest{
GroupID: groupID,
SessionTimeout: int32(time.Minute / time.Millisecond),
RebalanceTimeout: int32(time.Second / time.Millisecond),
Expand Down Expand Up @@ -770,7 +771,7 @@ func testConnFindCoordinator(t *testing.T, conn *Conn) {
}

func testConnJoinGroupInvalidGroupID(t *testing.T, conn *Conn) {
_, err := conn.joinGroup(joinGroupRequestV1{})
_, err := conn.joinGroup(joinGroupRequest{})
if !errors.Is(err, InvalidGroupId) && !errors.Is(err, NotCoordinatorForGroup) {
t.Fatalf("expected %v or %v; got %v", InvalidGroupId, NotCoordinatorForGroup, err)
}
Expand All @@ -780,7 +781,7 @@ func testConnJoinGroupInvalidSessionTimeout(t *testing.T, conn *Conn) {
groupID := makeGroupID()
waitForCoordinator(t, conn, groupID)

_, err := conn.joinGroup(joinGroupRequestV1{
_, err := conn.joinGroup(joinGroupRequest{
GroupID: groupID,
})
if !errors.Is(err, InvalidSessionTimeout) && !errors.Is(err, NotCoordinatorForGroup) {
Expand All @@ -792,7 +793,7 @@ func testConnJoinGroupInvalidRefreshTimeout(t *testing.T, conn *Conn) {
groupID := makeGroupID()
waitForCoordinator(t, conn, groupID)

_, err := conn.joinGroup(joinGroupRequestV1{
_, err := conn.joinGroup(joinGroupRequest{
GroupID: groupID,
SessionTimeout: int32(3 * time.Second / time.Millisecond),
})
Expand Down
18 changes: 9 additions & 9 deletions consumergroup.go
Original file line number Diff line number Diff line change
Expand Up @@ -555,7 +555,7 @@ func (g *Generation) partitionWatcher(interval time.Duration, topic string) {
type coordinator interface {
io.Closer
findCoordinator(findCoordinatorRequestV0) (findCoordinatorResponseV0, error)
joinGroup(joinGroupRequestV1) (joinGroupResponseV1, error)
joinGroup(joinGroupRequest) (joinGroupResponse, error)
syncGroup(syncGroupRequestV0) (syncGroupResponseV0, error)
leaveGroup(leaveGroupRequestV0) (leaveGroupResponseV0, error)
heartbeat(heartbeatRequestV0) (heartbeatResponseV0, error)
Expand Down Expand Up @@ -588,11 +588,11 @@ func (t *timeoutCoordinator) findCoordinator(req findCoordinatorRequestV0) (find
return t.conn.findCoordinator(req)
}

func (t *timeoutCoordinator) joinGroup(req joinGroupRequestV1) (joinGroupResponseV1, error) {
func (t *timeoutCoordinator) joinGroup(req joinGroupRequest) (joinGroupResponse, error) {
// in the case of join group, the consumer group coordinator may wait up
// to rebalance timeout in order to wait for all members to join.
if err := t.conn.SetDeadline(time.Now().Add(t.timeout + t.rebalanceTimeout)); err != nil {
return joinGroupResponseV1{}, err
return joinGroupResponse{}, err
}
return t.conn.joinGroup(req)
}
Expand Down Expand Up @@ -932,7 +932,7 @@ func (cg *ConsumerGroup) coordinator() (coordinator, error) {
// * InvalidSessionTimeout:
// * GroupAuthorizationFailed:
func (cg *ConsumerGroup) joinGroup(conn coordinator, memberID string) (string, int32, GroupMemberAssignments, error) {
request, err := cg.makeJoinGroupRequestV1(memberID)
request, err := cg.makeJoinGroupRequest(memberID)
if err != nil {
return "", 0, nil, err
}
Expand Down Expand Up @@ -978,8 +978,8 @@ func (cg *ConsumerGroup) joinGroup(conn coordinator, memberID string) (string, i

// makeJoinGroupRequestV1 handles the logic of constructing a joinGroup
// request.
func (cg *ConsumerGroup) makeJoinGroupRequestV1(memberID string) (joinGroupRequestV1, error) {
request := joinGroupRequestV1{
func (cg *ConsumerGroup) makeJoinGroupRequest(memberID string) (joinGroupRequest, error) {
request := joinGroupRequest{
GroupID: cg.config.ID,
MemberID: memberID,
SessionTimeout: int32(cg.config.SessionTimeout / time.Millisecond),
Expand All @@ -990,7 +990,7 @@ func (cg *ConsumerGroup) makeJoinGroupRequestV1(memberID string) (joinGroupReque
for _, balancer := range cg.config.GroupBalancers {
userData, err := balancer.UserData()
if err != nil {
return joinGroupRequestV1{}, fmt.Errorf("unable to construct protocol metadata for member, %v: %w", balancer.ProtocolName(), err)
return joinGroupRequest{}, fmt.Errorf("unable to construct protocol metadata for member, %v: %w", balancer.ProtocolName(), err)
}
request.GroupProtocols = append(request.GroupProtocols, joinGroupRequestGroupProtocolV1{
ProtocolName: balancer.ProtocolName(),
Expand All @@ -1007,7 +1007,7 @@ func (cg *ConsumerGroup) makeJoinGroupRequestV1(memberID string) (joinGroupReque

// assignTopicPartitions uses the selected GroupBalancer to assign members to
// their various partitions.
func (cg *ConsumerGroup) assignTopicPartitions(conn coordinator, group joinGroupResponseV1) (GroupMemberAssignments, error) {
func (cg *ConsumerGroup) assignTopicPartitions(conn coordinator, group joinGroupResponse) (GroupMemberAssignments, error) {
cg.withLogger(func(l Logger) {
l.Printf("selected as leader for group, %s\n", cg.config.ID)
})
Expand Down Expand Up @@ -1050,7 +1050,7 @@ func (cg *ConsumerGroup) assignTopicPartitions(conn coordinator, group joinGroup
}

// makeMemberProtocolMetadata maps encoded member metadata ([]byte) into []GroupMember.
func (cg *ConsumerGroup) makeMemberProtocolMetadata(in []joinGroupResponseMemberV1) ([]GroupMember, error) {
func (cg *ConsumerGroup) makeMemberProtocolMetadata(in []joinGroupResponseMember) ([]GroupMember, error) {
members := make([]GroupMember, 0, len(in))
for _, item := range in {
metadata := groupMetadata{}
Expand Down
Loading