github.com/QuangHoangHao/kafka-go@v0.4.36/joingroup.go (about)

     1  package kafka
     2  
     3  import (
     4  	"bufio"
     5  	"bytes"
     6  	"context"
     7  	"fmt"
     8  	"net"
     9  	"time"
    10  
    11  	"github.com/QuangHoangHao/kafka-go/protocol"
    12  	"github.com/QuangHoangHao/kafka-go/protocol/consumer"
    13  	"github.com/QuangHoangHao/kafka-go/protocol/joingroup"
    14  )
    15  
    16  // JoinGroupRequest is the request structure for the JoinGroup function.
    17  type JoinGroupRequest struct {
    18  	// Address of the kafka broker to send the request to.
    19  	Addr net.Addr
    20  
    21  	// GroupID of the group to join.
    22  	GroupID string
    23  
    24  	// The duration after which the coordinator considers the consumer dead
    25  	// if it has not received a heartbeat.
    26  	SessionTimeout time.Duration
    27  
    28  	// The duration the coordination will wait for each member to rejoin when rebalancing the group.
    29  	RebalanceTimeout time.Duration
    30  
    31  	// The ID assigned by the group coordinator.
    32  	MemberID string
    33  
    34  	// The unique identifier for the consumer instance.
    35  	GroupInstanceID string
    36  
    37  	// The name for the class of protocols implemented by the group being joined.
    38  	ProtocolType string
    39  
    40  	// The list of protocols the member supports.
    41  	Protocols []GroupProtocol
    42  }
    43  
    44  // GroupProtocol represents a consumer group protocol.
    45  type GroupProtocol struct {
    46  	// The protocol name.
    47  	Name string
    48  
    49  	// The protocol metadata.
    50  	Metadata GroupProtocolSubscription
    51  }
    52  
    53  type GroupProtocolSubscription struct {
    54  	// The Topics to subscribe to.
    55  	Topics []string
    56  
    57  	// UserData assosiated with the subscription for the given protocol
    58  	UserData []byte
    59  
    60  	// Partitions owned by this consumer.
    61  	OwnedPartitions map[string][]int
    62  }
    63  
    64  // JoinGroupResponse is the response structure for the JoinGroup function.
    65  type JoinGroupResponse struct {
    66  	// An error that may have occurred when attempting to join the group.
    67  	//
    68  	// The errors contain the kafka error code. Programs may use the standard
    69  	// errors.Is function to test the error against kafka error codes.
    70  	Error error
    71  
    72  	// The amount of time that the broker throttled the request.
    73  	Throttle time.Duration
    74  
    75  	// The generation ID of the group.
    76  	GenerationID int
    77  
    78  	// The group protocol selected by the coordinatior.
    79  	ProtocolName string
    80  
    81  	// The group protocol name.
    82  	ProtocolType string
    83  
    84  	// The leader of the group.
    85  	LeaderID string
    86  
    87  	// The group member ID.
    88  	MemberID string
    89  
    90  	// The members of the group.
    91  	Members []JoinGroupResponseMember
    92  }
    93  
    94  // JoinGroupResponseMember represents a group memmber in a reponse to a JoinGroup request.
    95  type JoinGroupResponseMember struct {
    96  	// The group memmber ID.
    97  	ID string
    98  
    99  	// The unique identifier of the consumer instance.
   100  	GroupInstanceID string
   101  
   102  	// The group member metadata.
   103  	Metadata GroupProtocolSubscription
   104  }
   105  
   106  // JoinGroup sends a join group request to the coordinator and returns the response.
   107  func (c *Client) JoinGroup(ctx context.Context, req *JoinGroupRequest) (*JoinGroupResponse, error) {
   108  	joinGroup := joingroup.Request{
   109  		GroupID:            req.GroupID,
   110  		SessionTimeoutMS:   int32(req.SessionTimeout.Milliseconds()),
   111  		RebalanceTimeoutMS: int32(req.RebalanceTimeout.Milliseconds()),
   112  		MemberID:           req.MemberID,
   113  		GroupInstanceID:    req.GroupInstanceID,
   114  		ProtocolType:       req.ProtocolType,
   115  		Protocols:          make([]joingroup.RequestProtocol, 0, len(req.Protocols)),
   116  	}
   117  
   118  	for _, proto := range req.Protocols {
   119  		protoMeta := consumer.Subscription{
   120  			Version:         consumer.MaxVersionSupported,
   121  			Topics:          proto.Metadata.Topics,
   122  			UserData:        proto.Metadata.UserData,
   123  			OwnedPartitions: make([]consumer.TopicPartition, 0, len(proto.Metadata.OwnedPartitions)),
   124  		}
   125  		for topic, partitions := range proto.Metadata.OwnedPartitions {
   126  			tp := consumer.TopicPartition{
   127  				Topic:      topic,
   128  				Partitions: make([]int32, 0, len(partitions)),
   129  			}
   130  			for _, partition := range partitions {
   131  				tp.Partitions = append(tp.Partitions, int32(partition))
   132  			}
   133  			protoMeta.OwnedPartitions = append(protoMeta.OwnedPartitions, tp)
   134  		}
   135  
   136  		metaBytes, err := protocol.Marshal(consumer.MaxVersionSupported, protoMeta)
   137  		if err != nil {
   138  			return nil, fmt.Errorf("kafka.(*Client).JoinGroup: %w", err)
   139  		}
   140  
   141  		joinGroup.Protocols = append(joinGroup.Protocols, joingroup.RequestProtocol{
   142  			Name:     proto.Name,
   143  			Metadata: metaBytes,
   144  		})
   145  	}
   146  
   147  	m, err := c.roundTrip(ctx, req.Addr, &joinGroup)
   148  	if err != nil {
   149  		return nil, fmt.Errorf("kafka.(*Client).JoinGroup: %w", err)
   150  	}
   151  
   152  	r := m.(*joingroup.Response)
   153  
   154  	res := &JoinGroupResponse{
   155  		Error:        makeError(r.ErrorCode, ""),
   156  		Throttle:     makeDuration(r.ThrottleTimeMS),
   157  		GenerationID: int(r.GenerationID),
   158  		ProtocolName: r.ProtocolName,
   159  		ProtocolType: r.ProtocolType,
   160  		LeaderID:     r.LeaderID,
   161  		MemberID:     r.MemberID,
   162  		Members:      make([]JoinGroupResponseMember, 0, len(r.Members)),
   163  	}
   164  
   165  	for _, member := range r.Members {
   166  		var meta consumer.Subscription
   167  		err = protocol.Unmarshal(member.Metadata, consumer.MaxVersionSupported, &meta)
   168  		if err != nil {
   169  			return nil, fmt.Errorf("kafka.(*Client).JoinGroup: %w", err)
   170  		}
   171  		subscription := GroupProtocolSubscription{
   172  			Topics:          meta.Topics,
   173  			UserData:        meta.UserData,
   174  			OwnedPartitions: make(map[string][]int, len(meta.OwnedPartitions)),
   175  		}
   176  		for _, owned := range meta.OwnedPartitions {
   177  			subscription.OwnedPartitions[owned.Topic] = make([]int, 0, len(owned.Partitions))
   178  			for _, partition := range owned.Partitions {
   179  				subscription.OwnedPartitions[owned.Topic] = append(subscription.OwnedPartitions[owned.Topic], int(partition))
   180  			}
   181  		}
   182  		res.Members = append(res.Members, JoinGroupResponseMember{
   183  			ID:              member.MemberID,
   184  			GroupInstanceID: member.GroupInstanceID,
   185  			Metadata:        subscription,
   186  		})
   187  	}
   188  
   189  	return res, nil
   190  }
   191  
   192  type groupMetadata struct {
   193  	Version  int16
   194  	Topics   []string
   195  	UserData []byte
   196  }
   197  
   198  func (t groupMetadata) size() int32 {
   199  	return sizeofInt16(t.Version) +
   200  		sizeofStringArray(t.Topics) +
   201  		sizeofBytes(t.UserData)
   202  }
   203  
   204  func (t groupMetadata) writeTo(wb *writeBuffer) {
   205  	wb.writeInt16(t.Version)
   206  	wb.writeStringArray(t.Topics)
   207  	wb.writeBytes(t.UserData)
   208  }
   209  
   210  func (t groupMetadata) bytes() []byte {
   211  	buf := bytes.NewBuffer(nil)
   212  	t.writeTo(&writeBuffer{w: buf})
   213  	return buf.Bytes()
   214  }
   215  
   216  func (t *groupMetadata) readFrom(r *bufio.Reader, size int) (remain int, err error) {
   217  	if remain, err = readInt16(r, size, &t.Version); err != nil {
   218  		return
   219  	}
   220  	if remain, err = readStringArray(r, remain, &t.Topics); err != nil {
   221  		return
   222  	}
   223  	if remain, err = readBytes(r, remain, &t.UserData); err != nil {
   224  		return
   225  	}
   226  	return
   227  }
   228  
   229  type joinGroupRequestGroupProtocolV1 struct {
   230  	ProtocolName     string
   231  	ProtocolMetadata []byte
   232  }
   233  
   234  func (t joinGroupRequestGroupProtocolV1) size() int32 {
   235  	return sizeofString(t.ProtocolName) +
   236  		sizeofBytes(t.ProtocolMetadata)
   237  }
   238  
   239  func (t joinGroupRequestGroupProtocolV1) writeTo(wb *writeBuffer) {
   240  	wb.writeString(t.ProtocolName)
   241  	wb.writeBytes(t.ProtocolMetadata)
   242  }
   243  
   244  type joinGroupRequestV1 struct {
   245  	// GroupID holds the unique group identifier
   246  	GroupID string
   247  
   248  	// SessionTimeout holds the coordinator considers the consumer dead if it
   249  	// receives no heartbeat after this timeout in ms.
   250  	SessionTimeout int32
   251  
   252  	// RebalanceTimeout holds the maximum time that the coordinator will wait
   253  	// for each member to rejoin when rebalancing the group in ms
   254  	RebalanceTimeout int32
   255  
   256  	// MemberID assigned by the group coordinator or the zero string if joining
   257  	// for the first time.
   258  	MemberID string
   259  
   260  	// ProtocolType holds the unique name for class of protocols implemented by group
   261  	ProtocolType string
   262  
   263  	// GroupProtocols holds the list of protocols that the member supports
   264  	GroupProtocols []joinGroupRequestGroupProtocolV1
   265  }
   266  
   267  func (t joinGroupRequestV1) size() int32 {
   268  	return sizeofString(t.GroupID) +
   269  		sizeofInt32(t.SessionTimeout) +
   270  		sizeofInt32(t.RebalanceTimeout) +
   271  		sizeofString(t.MemberID) +
   272  		sizeofString(t.ProtocolType) +
   273  		sizeofArray(len(t.GroupProtocols), func(i int) int32 { return t.GroupProtocols[i].size() })
   274  }
   275  
   276  func (t joinGroupRequestV1) writeTo(wb *writeBuffer) {
   277  	wb.writeString(t.GroupID)
   278  	wb.writeInt32(t.SessionTimeout)
   279  	wb.writeInt32(t.RebalanceTimeout)
   280  	wb.writeString(t.MemberID)
   281  	wb.writeString(t.ProtocolType)
   282  	wb.writeArray(len(t.GroupProtocols), func(i int) { t.GroupProtocols[i].writeTo(wb) })
   283  }
   284  
   285  type joinGroupResponseMemberV1 struct {
   286  	// MemberID assigned by the group coordinator
   287  	MemberID       string
   288  	MemberMetadata []byte
   289  }
   290  
   291  func (t joinGroupResponseMemberV1) size() int32 {
   292  	return sizeofString(t.MemberID) +
   293  		sizeofBytes(t.MemberMetadata)
   294  }
   295  
   296  func (t joinGroupResponseMemberV1) writeTo(wb *writeBuffer) {
   297  	wb.writeString(t.MemberID)
   298  	wb.writeBytes(t.MemberMetadata)
   299  }
   300  
   301  func (t *joinGroupResponseMemberV1) readFrom(r *bufio.Reader, size int) (remain int, err error) {
   302  	if remain, err = readString(r, size, &t.MemberID); err != nil {
   303  		return
   304  	}
   305  	if remain, err = readBytes(r, remain, &t.MemberMetadata); err != nil {
   306  		return
   307  	}
   308  	return
   309  }
   310  
   311  type joinGroupResponseV1 struct {
   312  	// ErrorCode holds response error code
   313  	ErrorCode int16
   314  
   315  	// GenerationID holds the generation of the group.
   316  	GenerationID int32
   317  
   318  	// GroupProtocol holds the group protocol selected by the coordinator
   319  	GroupProtocol string
   320  
   321  	// LeaderID holds the leader of the group
   322  	LeaderID string
   323  
   324  	// MemberID assigned by the group coordinator
   325  	MemberID string
   326  	Members  []joinGroupResponseMemberV1
   327  }
   328  
   329  func (t joinGroupResponseV1) size() int32 {
   330  	return sizeofInt16(t.ErrorCode) +
   331  		sizeofInt32(t.GenerationID) +
   332  		sizeofString(t.GroupProtocol) +
   333  		sizeofString(t.LeaderID) +
   334  		sizeofString(t.MemberID) +
   335  		sizeofArray(len(t.MemberID), func(i int) int32 { return t.Members[i].size() })
   336  }
   337  
   338  func (t joinGroupResponseV1) writeTo(wb *writeBuffer) {
   339  	wb.writeInt16(t.ErrorCode)
   340  	wb.writeInt32(t.GenerationID)
   341  	wb.writeString(t.GroupProtocol)
   342  	wb.writeString(t.LeaderID)
   343  	wb.writeString(t.MemberID)
   344  	wb.writeArray(len(t.Members), func(i int) { t.Members[i].writeTo(wb) })
   345  }
   346  
   347  func (t *joinGroupResponseV1) readFrom(r *bufio.Reader, size int) (remain int, err error) {
   348  	if remain, err = readInt16(r, size, &t.ErrorCode); err != nil {
   349  		return
   350  	}
   351  	if remain, err = readInt32(r, remain, &t.GenerationID); err != nil {
   352  		return
   353  	}
   354  	if remain, err = readString(r, remain, &t.GroupProtocol); err != nil {
   355  		return
   356  	}
   357  	if remain, err = readString(r, remain, &t.LeaderID); err != nil {
   358  		return
   359  	}
   360  	if remain, err = readString(r, remain, &t.MemberID); err != nil {
   361  		return
   362  	}
   363  
   364  	fn := func(r *bufio.Reader, size int) (fnRemain int, fnErr error) {
   365  		var item joinGroupResponseMemberV1
   366  		if fnRemain, fnErr = (&item).readFrom(r, size); fnErr != nil {
   367  			return
   368  		}
   369  		t.Members = append(t.Members, item)
   370  		return
   371  	}
   372  	if remain, err = readArrayWith(r, remain, fn); err != nil {
   373  		return
   374  	}
   375  
   376  	return
   377  }