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

     1  package kafka
     2  
     3  import (
     4  	"bufio"
     5  	"context"
     6  	"fmt"
     7  	"net"
     8  	"time"
     9  
    10  	"github.com/QuangHoangHao/kafka-go/protocol/listoffsets"
    11  )
    12  
    13  // OffsetRequest represents a request to retrieve a single partition offset.
    14  type OffsetRequest struct {
    15  	Partition int
    16  	Timestamp int64
    17  }
    18  
    19  // FirstOffsetOf constructs an OffsetRequest which asks for the first offset of
    20  // the parition given as argument.
    21  func FirstOffsetOf(partition int) OffsetRequest {
    22  	return OffsetRequest{Partition: partition, Timestamp: FirstOffset}
    23  }
    24  
    25  // LastOffsetOf constructs an OffsetRequest which asks for the last offset of
    26  // the partition given as argument.
    27  func LastOffsetOf(partition int) OffsetRequest {
    28  	return OffsetRequest{Partition: partition, Timestamp: LastOffset}
    29  }
    30  
    31  // TimeOffsetOf constructs an OffsetRequest which asks for a partition offset
    32  // at a given time.
    33  func TimeOffsetOf(partition int, at time.Time) OffsetRequest {
    34  	return OffsetRequest{Partition: partition, Timestamp: timestamp(at)}
    35  }
    36  
    37  // PartitionOffsets carries information about offsets available in a topic
    38  // partition.
    39  type PartitionOffsets struct {
    40  	Partition   int
    41  	FirstOffset int64
    42  	LastOffset  int64
    43  	Offsets     map[int64]time.Time
    44  	Error       error
    45  }
    46  
    47  // ListOffsetsRequest represents a request sent to a kafka broker to list of the
    48  // offsets of topic partitions.
    49  type ListOffsetsRequest struct {
    50  	// Address of the kafka broker to send the request to.
    51  	Addr net.Addr
    52  
    53  	// A mapping of topic names to list of partitions that the program wishes to
    54  	// get the offsets for.
    55  	Topics map[string][]OffsetRequest
    56  
    57  	// The isolation level for the request.
    58  	//
    59  	// Defaults to ReadUncommitted.
    60  	//
    61  	// This field requires the kafka broker to support the ListOffsets API in
    62  	// version 2 or above (otherwise the value is ignored).
    63  	IsolationLevel IsolationLevel
    64  }
    65  
    66  // ListOffsetsResponse represents a response from a kafka broker to a offset
    67  // listing request.
    68  type ListOffsetsResponse struct {
    69  	// The amount of time that the broker throttled the request.
    70  	Throttle time.Duration
    71  
    72  	// Mappings of topics names to partition offsets, there will be one entry
    73  	// for each topic in the request.
    74  	Topics map[string][]PartitionOffsets
    75  }
    76  
    77  // ListOffsets sends an offset request to a kafka broker and returns the
    78  // response.
    79  func (c *Client) ListOffsets(ctx context.Context, req *ListOffsetsRequest) (*ListOffsetsResponse, error) {
    80  	type topicPartition struct {
    81  		topic     string
    82  		partition int
    83  	}
    84  
    85  	partitionOffsets := make(map[topicPartition]PartitionOffsets)
    86  
    87  	for topicName, requests := range req.Topics {
    88  		for _, r := range requests {
    89  			key := topicPartition{
    90  				topic:     topicName,
    91  				partition: r.Partition,
    92  			}
    93  
    94  			partition, ok := partitionOffsets[key]
    95  			if !ok {
    96  				partition = PartitionOffsets{
    97  					Partition:   r.Partition,
    98  					FirstOffset: -1,
    99  					LastOffset:  -1,
   100  					Offsets:     make(map[int64]time.Time),
   101  				}
   102  			}
   103  
   104  			switch r.Timestamp {
   105  			case FirstOffset:
   106  				partition.FirstOffset = 0
   107  			case LastOffset:
   108  				partition.LastOffset = 0
   109  			}
   110  
   111  			partitionOffsets[topicPartition{
   112  				topic:     topicName,
   113  				partition: r.Partition,
   114  			}] = partition
   115  		}
   116  	}
   117  
   118  	topics := make([]listoffsets.RequestTopic, 0, len(req.Topics))
   119  
   120  	for topicName, requests := range req.Topics {
   121  		partitions := make([]listoffsets.RequestPartition, len(requests))
   122  
   123  		for i, r := range requests {
   124  			partitions[i] = listoffsets.RequestPartition{
   125  				Partition:          int32(r.Partition),
   126  				CurrentLeaderEpoch: -1,
   127  				Timestamp:          r.Timestamp,
   128  			}
   129  		}
   130  
   131  		topics = append(topics, listoffsets.RequestTopic{
   132  			Topic:      topicName,
   133  			Partitions: partitions,
   134  		})
   135  	}
   136  
   137  	m, err := c.roundTrip(ctx, req.Addr, &listoffsets.Request{
   138  		ReplicaID:      -1,
   139  		IsolationLevel: int8(req.IsolationLevel),
   140  		Topics:         topics,
   141  	})
   142  
   143  	if err != nil {
   144  		return nil, fmt.Errorf("kafka.(*Client).ListOffsets: %w", err)
   145  	}
   146  
   147  	res := m.(*listoffsets.Response)
   148  	ret := &ListOffsetsResponse{
   149  		Throttle: makeDuration(res.ThrottleTimeMs),
   150  		Topics:   make(map[string][]PartitionOffsets, len(res.Topics)),
   151  	}
   152  
   153  	for _, t := range res.Topics {
   154  		for _, p := range t.Partitions {
   155  			key := topicPartition{
   156  				topic:     t.Topic,
   157  				partition: int(p.Partition),
   158  			}
   159  
   160  			partition := partitionOffsets[key]
   161  
   162  			switch p.Timestamp {
   163  			case FirstOffset:
   164  				partition.FirstOffset = p.Offset
   165  			case LastOffset:
   166  				partition.LastOffset = p.Offset
   167  			default:
   168  				partition.Offsets[p.Offset] = makeTime(p.Timestamp)
   169  			}
   170  
   171  			if p.ErrorCode != 0 {
   172  				partition.Error = Error(p.ErrorCode)
   173  			}
   174  
   175  			partitionOffsets[key] = partition
   176  		}
   177  	}
   178  
   179  	for key, partition := range partitionOffsets {
   180  		ret.Topics[key.topic] = append(ret.Topics[key.topic], partition)
   181  	}
   182  
   183  	return ret, nil
   184  }
   185  
   186  type listOffsetRequestV1 struct {
   187  	ReplicaID int32
   188  	Topics    []listOffsetRequestTopicV1
   189  }
   190  
   191  func (r listOffsetRequestV1) size() int32 {
   192  	return 4 + sizeofArray(len(r.Topics), func(i int) int32 { return r.Topics[i].size() })
   193  }
   194  
   195  func (r listOffsetRequestV1) writeTo(wb *writeBuffer) {
   196  	wb.writeInt32(r.ReplicaID)
   197  	wb.writeArray(len(r.Topics), func(i int) { r.Topics[i].writeTo(wb) })
   198  }
   199  
   200  type listOffsetRequestTopicV1 struct {
   201  	TopicName  string
   202  	Partitions []listOffsetRequestPartitionV1
   203  }
   204  
   205  func (t listOffsetRequestTopicV1) size() int32 {
   206  	return sizeofString(t.TopicName) +
   207  		sizeofArray(len(t.Partitions), func(i int) int32 { return t.Partitions[i].size() })
   208  }
   209  
   210  func (t listOffsetRequestTopicV1) writeTo(wb *writeBuffer) {
   211  	wb.writeString(t.TopicName)
   212  	wb.writeArray(len(t.Partitions), func(i int) { t.Partitions[i].writeTo(wb) })
   213  }
   214  
   215  type listOffsetRequestPartitionV1 struct {
   216  	Partition int32
   217  	Time      int64
   218  }
   219  
   220  func (p listOffsetRequestPartitionV1) size() int32 {
   221  	return 4 + 8
   222  }
   223  
   224  func (p listOffsetRequestPartitionV1) writeTo(wb *writeBuffer) {
   225  	wb.writeInt32(p.Partition)
   226  	wb.writeInt64(p.Time)
   227  }
   228  
   229  type listOffsetResponseV1 []listOffsetResponseTopicV1
   230  
   231  func (r listOffsetResponseV1) size() int32 {
   232  	return sizeofArray(len(r), func(i int) int32 { return r[i].size() })
   233  }
   234  
   235  func (r listOffsetResponseV1) writeTo(wb *writeBuffer) {
   236  	wb.writeArray(len(r), func(i int) { r[i].writeTo(wb) })
   237  }
   238  
   239  type listOffsetResponseTopicV1 struct {
   240  	TopicName        string
   241  	PartitionOffsets []partitionOffsetV1
   242  }
   243  
   244  func (t listOffsetResponseTopicV1) size() int32 {
   245  	return sizeofString(t.TopicName) +
   246  		sizeofArray(len(t.PartitionOffsets), func(i int) int32 { return t.PartitionOffsets[i].size() })
   247  }
   248  
   249  func (t listOffsetResponseTopicV1) writeTo(wb *writeBuffer) {
   250  	wb.writeString(t.TopicName)
   251  	wb.writeArray(len(t.PartitionOffsets), func(i int) { t.PartitionOffsets[i].writeTo(wb) })
   252  }
   253  
   254  type partitionOffsetV1 struct {
   255  	Partition int32
   256  	ErrorCode int16
   257  	Timestamp int64
   258  	Offset    int64
   259  }
   260  
   261  func (p partitionOffsetV1) size() int32 {
   262  	return 4 + 2 + 8 + 8
   263  }
   264  
   265  func (p partitionOffsetV1) writeTo(wb *writeBuffer) {
   266  	wb.writeInt32(p.Partition)
   267  	wb.writeInt16(p.ErrorCode)
   268  	wb.writeInt64(p.Timestamp)
   269  	wb.writeInt64(p.Offset)
   270  }
   271  
   272  func (p *partitionOffsetV1) readFrom(r *bufio.Reader, sz int) (remain int, err error) {
   273  	if remain, err = readInt32(r, sz, &p.Partition); err != nil {
   274  		return
   275  	}
   276  	if remain, err = readInt16(r, remain, &p.ErrorCode); err != nil {
   277  		return
   278  	}
   279  	if remain, err = readInt64(r, remain, &p.Timestamp); err != nil {
   280  		return
   281  	}
   282  	if remain, err = readInt64(r, remain, &p.Offset); err != nil {
   283  		return
   284  	}
   285  	return
   286  }