github.com/segmentio/kafka-go@v0.4.48-0.20240318174348-3f6244eb34fd/heartbeat.go (about)

     1  package kafka
     2  
     3  import (
     4  	"bufio"
     5  	"context"
     6  	"fmt"
     7  	"net"
     8  	"time"
     9  
    10  	heartbeatAPI "github.com/segmentio/kafka-go/protocol/heartbeat"
    11  )
    12  
    13  // HeartbeatRequest represents a heartbeat sent to kafka to indicate consume liveness.
    14  type HeartbeatRequest struct {
    15  	// Address of the kafka broker to send the request to.
    16  	Addr net.Addr
    17  
    18  	// GroupID is the ID of the group.
    19  	GroupID string
    20  
    21  	// GenerationID is the current generation for the group.
    22  	GenerationID int32
    23  
    24  	// MemberID is the ID of the group member.
    25  	MemberID string
    26  
    27  	// GroupInstanceID is a unique identifier for the consumer.
    28  	GroupInstanceID string
    29  }
    30  
    31  // HeartbeatResponse represents a response from a heartbeat request.
    32  type HeartbeatResponse struct {
    33  	// Error is set to non-nil if an error occurred.
    34  	Error error
    35  
    36  	// The amount of time that the broker throttled the request.
    37  	//
    38  	// This field will be zero if the kafka broker did not support the
    39  	// Heartbeat API in version 1 or above.
    40  	Throttle time.Duration
    41  }
    42  
    43  type heartbeatRequestV0 struct {
    44  	// GroupID holds the unique group identifier
    45  	GroupID string
    46  
    47  	// GenerationID holds the generation of the group.
    48  	GenerationID int32
    49  
    50  	// MemberID assigned by the group coordinator
    51  	MemberID string
    52  }
    53  
    54  // Heartbeat sends a heartbeat request to a kafka broker and returns the response.
    55  func (c *Client) Heartbeat(ctx context.Context, req *HeartbeatRequest) (*HeartbeatResponse, error) {
    56  	m, err := c.roundTrip(ctx, req.Addr, &heartbeatAPI.Request{
    57  		GroupID:         req.GroupID,
    58  		GenerationID:    req.GenerationID,
    59  		MemberID:        req.MemberID,
    60  		GroupInstanceID: req.GroupInstanceID,
    61  	})
    62  	if err != nil {
    63  		return nil, fmt.Errorf("kafka.(*Client).Heartbeat: %w", err)
    64  	}
    65  
    66  	res := m.(*heartbeatAPI.Response)
    67  
    68  	ret := &HeartbeatResponse{
    69  		Throttle: makeDuration(res.ThrottleTimeMs),
    70  	}
    71  
    72  	if res.ErrorCode != 0 {
    73  		ret.Error = Error(res.ErrorCode)
    74  	}
    75  
    76  	return ret, nil
    77  }
    78  
    79  func (t heartbeatRequestV0) size() int32 {
    80  	return sizeofString(t.GroupID) +
    81  		sizeofInt32(t.GenerationID) +
    82  		sizeofString(t.MemberID)
    83  }
    84  
    85  func (t heartbeatRequestV0) writeTo(wb *writeBuffer) {
    86  	wb.writeString(t.GroupID)
    87  	wb.writeInt32(t.GenerationID)
    88  	wb.writeString(t.MemberID)
    89  }
    90  
    91  type heartbeatResponseV0 struct {
    92  	// ErrorCode holds response error code
    93  	ErrorCode int16
    94  }
    95  
    96  func (t heartbeatResponseV0) size() int32 {
    97  	return sizeofInt16(t.ErrorCode)
    98  }
    99  
   100  func (t heartbeatResponseV0) writeTo(wb *writeBuffer) {
   101  	wb.writeInt16(t.ErrorCode)
   102  }
   103  
   104  func (t *heartbeatResponseV0) readFrom(r *bufio.Reader, sz int) (remain int, err error) {
   105  	if remain, err = readInt16(r, sz, &t.ErrorCode); err != nil {
   106  		return
   107  	}
   108  	return
   109  }