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

     1  package kafka
     2  
     3  import (
     4  	"context"
     5  	"fmt"
     6  	"net"
     7  	"time"
     8  
     9  	"github.com/segmentio/kafka-go/protocol/offsetdelete"
    10  )
    11  
    12  // OffsetDelete deletes the offset for a consumer group on a particular topic
    13  // for a particular partition.
    14  type OffsetDelete struct {
    15  	Topic     string
    16  	Partition int
    17  }
    18  
    19  // OffsetDeleteRequest represents a request sent to a kafka broker to delete
    20  // the offsets for a partition on a given topic associated with a consumer group.
    21  type OffsetDeleteRequest struct {
    22  	// Address of the kafka broker to send the request to.
    23  	Addr net.Addr
    24  
    25  	// ID of the consumer group to delete the offsets for.
    26  	GroupID string
    27  
    28  	// Set of topic partitions to delete offsets for.
    29  	Topics map[string][]int
    30  }
    31  
    32  // OffsetDeleteResponse represents a response from a kafka broker to a delete
    33  // offset request.
    34  type OffsetDeleteResponse struct {
    35  	// An error that may have occurred while attempting to delete an offset
    36  	Error error
    37  
    38  	// The amount of time that the broker throttled the request.
    39  	Throttle time.Duration
    40  
    41  	// Set of topic partitions that the kafka broker has additional info (error?)
    42  	// for.
    43  	Topics map[string][]OffsetDeletePartition
    44  }
    45  
    46  // OffsetDeletePartition represents the state of a status of a partition in response
    47  // to deleting offsets.
    48  type OffsetDeletePartition struct {
    49  	// ID of the partition.
    50  	Partition int
    51  
    52  	// An error that may have occurred while attempting to delete an offset for
    53  	// this partition.
    54  	Error error
    55  }
    56  
    57  // OffsetDelete sends a delete offset request to a kafka broker and returns the
    58  // response.
    59  func (c *Client) OffsetDelete(ctx context.Context, req *OffsetDeleteRequest) (*OffsetDeleteResponse, error) {
    60  	topics := make([]offsetdelete.RequestTopic, 0, len(req.Topics))
    61  
    62  	for topicName, partitionIndexes := range req.Topics {
    63  		partitions := make([]offsetdelete.RequestPartition, len(partitionIndexes))
    64  
    65  		for i, c := range partitionIndexes {
    66  			partitions[i] = offsetdelete.RequestPartition{
    67  				PartitionIndex: int32(c),
    68  			}
    69  		}
    70  
    71  		topics = append(topics, offsetdelete.RequestTopic{
    72  			Name:       topicName,
    73  			Partitions: partitions,
    74  		})
    75  	}
    76  
    77  	m, err := c.roundTrip(ctx, req.Addr, &offsetdelete.Request{
    78  		GroupID: req.GroupID,
    79  		Topics:  topics,
    80  	})
    81  	if err != nil {
    82  		return nil, fmt.Errorf("kafka.(*Client).OffsetDelete: %w", err)
    83  	}
    84  	r := m.(*offsetdelete.Response)
    85  
    86  	res := &OffsetDeleteResponse{
    87  		Error:    makeError(r.ErrorCode, ""),
    88  		Throttle: makeDuration(r.ThrottleTimeMs),
    89  		Topics:   make(map[string][]OffsetDeletePartition, len(r.Topics)),
    90  	}
    91  
    92  	for _, topic := range r.Topics {
    93  		partitions := make([]OffsetDeletePartition, len(topic.Partitions))
    94  
    95  		for i, p := range topic.Partitions {
    96  			partitions[i] = OffsetDeletePartition{
    97  				Partition: int(p.PartitionIndex),
    98  				Error:     makeError(p.ErrorCode, ""),
    99  			}
   100  		}
   101  
   102  		res.Topics[topic.Name] = partitions
   103  	}
   104  
   105  	return res, nil
   106  }