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

     1  package kafka
     2  
     3  import (
     4  	"bufio"
     5  	"context"
     6  	"fmt"
     7  	"net"
     8  	"time"
     9  
    10  	"github.com/segmentio/kafka-go/protocol/deletetopics"
    11  )
    12  
    13  // DeleteTopicsRequest represents a request sent to a kafka broker to delete
    14  // topics.
    15  type DeleteTopicsRequest struct {
    16  	// Address of the kafka broker to send the request to.
    17  	Addr net.Addr
    18  
    19  	// Names of topics to delete.
    20  	Topics []string
    21  }
    22  
    23  // DeleteTopicsResponse represents a response from a kafka broker to a topic
    24  // deletion request.
    25  type DeleteTopicsResponse struct {
    26  	// The amount of time that the broker throttled the request.
    27  	//
    28  	// This field will be zero if the kafka broker did not support the
    29  	// DeleteTopics API in version 1 or above.
    30  	Throttle time.Duration
    31  
    32  	// Mapping of topic names to errors that occurred while attempting to delete
    33  	// the topics.
    34  	//
    35  	// The errors contain the kafka error code. Programs may use the standard
    36  	// errors.Is function to test the error against kafka error codes.
    37  	Errors map[string]error
    38  }
    39  
    40  // DeleteTopics sends a topic deletion request to a kafka broker and returns the
    41  // response.
    42  func (c *Client) DeleteTopics(ctx context.Context, req *DeleteTopicsRequest) (*DeleteTopicsResponse, error) {
    43  	m, err := c.roundTrip(ctx, req.Addr, &deletetopics.Request{
    44  		TopicNames: req.Topics,
    45  		TimeoutMs:  c.timeoutMs(ctx, defaultDeleteTopicsTimeout),
    46  	})
    47  
    48  	if err != nil {
    49  		return nil, fmt.Errorf("kafka.(*Client).DeleteTopics: %w", err)
    50  	}
    51  
    52  	res := m.(*deletetopics.Response)
    53  	ret := &DeleteTopicsResponse{
    54  		Throttle: makeDuration(res.ThrottleTimeMs),
    55  		Errors:   make(map[string]error, len(res.Responses)),
    56  	}
    57  
    58  	for _, t := range res.Responses {
    59  		if t.ErrorCode == 0 {
    60  			ret.Errors[t.Name] = nil
    61  		} else {
    62  			ret.Errors[t.Name] = Error(t.ErrorCode)
    63  		}
    64  	}
    65  
    66  	return ret, nil
    67  }
    68  
    69  // See http://kafka.apache.org/protocol.html#The_Messages_DeleteTopics
    70  type deleteTopicsRequestV0 struct {
    71  	// Topics holds the topic names
    72  	Topics []string
    73  
    74  	// Timeout holds the time in ms to wait for a topic to be completely deleted
    75  	// on the controller node. Values <= 0 will trigger topic deletion and return
    76  	// immediately.
    77  	Timeout int32
    78  }
    79  
    80  func (t deleteTopicsRequestV0) size() int32 {
    81  	return sizeofStringArray(t.Topics) +
    82  		sizeofInt32(t.Timeout)
    83  }
    84  
    85  func (t deleteTopicsRequestV0) writeTo(wb *writeBuffer) {
    86  	wb.writeStringArray(t.Topics)
    87  	wb.writeInt32(t.Timeout)
    88  }
    89  
    90  type deleteTopicsResponseV0 struct {
    91  	// TopicErrorCodes holds per topic error codes
    92  	TopicErrorCodes []deleteTopicsResponseV0TopicErrorCode
    93  }
    94  
    95  func (t deleteTopicsResponseV0) size() int32 {
    96  	return sizeofArray(len(t.TopicErrorCodes), func(i int) int32 { return t.TopicErrorCodes[i].size() })
    97  }
    98  
    99  func (t *deleteTopicsResponseV0) readFrom(r *bufio.Reader, size int) (remain int, err error) {
   100  	fn := func(withReader *bufio.Reader, withSize int) (fnRemain int, fnErr error) {
   101  		var item deleteTopicsResponseV0TopicErrorCode
   102  		if fnRemain, fnErr = (&item).readFrom(withReader, withSize); err != nil {
   103  			return
   104  		}
   105  		t.TopicErrorCodes = append(t.TopicErrorCodes, item)
   106  		return
   107  	}
   108  	if remain, err = readArrayWith(r, size, fn); err != nil {
   109  		return
   110  	}
   111  	return
   112  }
   113  
   114  func (t deleteTopicsResponseV0) writeTo(wb *writeBuffer) {
   115  	wb.writeArray(len(t.TopicErrorCodes), func(i int) { t.TopicErrorCodes[i].writeTo(wb) })
   116  }
   117  
   118  type deleteTopicsResponseV0TopicErrorCode struct {
   119  	// Topic holds the topic name
   120  	Topic string
   121  
   122  	// ErrorCode holds the error code
   123  	ErrorCode int16
   124  }
   125  
   126  func (t deleteTopicsResponseV0TopicErrorCode) size() int32 {
   127  	return sizeofString(t.Topic) +
   128  		sizeofInt16(t.ErrorCode)
   129  }
   130  
   131  func (t *deleteTopicsResponseV0TopicErrorCode) readFrom(r *bufio.Reader, size int) (remain int, err error) {
   132  	if remain, err = readString(r, size, &t.Topic); err != nil {
   133  		return
   134  	}
   135  	if remain, err = readInt16(r, remain, &t.ErrorCode); err != nil {
   136  		return
   137  	}
   138  	return
   139  }
   140  
   141  func (t deleteTopicsResponseV0TopicErrorCode) writeTo(wb *writeBuffer) {
   142  	wb.writeString(t.Topic)
   143  	wb.writeInt16(t.ErrorCode)
   144  }
   145  
   146  // deleteTopics deletes the specified topics.
   147  //
   148  // See http://kafka.apache.org/protocol.html#The_Messages_DeleteTopics
   149  func (c *Conn) deleteTopics(request deleteTopicsRequestV0) (deleteTopicsResponseV0, error) {
   150  	var response deleteTopicsResponseV0
   151  	err := c.writeOperation(
   152  		func(deadline time.Time, id int32) error {
   153  			if request.Timeout == 0 {
   154  				now := time.Now()
   155  				deadline = adjustDeadlineForRTT(deadline, now, defaultRTT)
   156  				request.Timeout = milliseconds(deadlineToTimeout(deadline, now))
   157  			}
   158  			return c.writeRequest(deleteTopics, v0, id, request)
   159  		},
   160  		func(deadline time.Time, size int) error {
   161  			return expectZeroSize(func() (remain int, err error) {
   162  				return (&response).readFrom(&c.rbuf, size)
   163  			}())
   164  		},
   165  	)
   166  	if err != nil {
   167  		return deleteTopicsResponseV0{}, err
   168  	}
   169  	for _, c := range response.TopicErrorCodes {
   170  		if c.ErrorCode != 0 {
   171  			return response, Error(c.ErrorCode)
   172  		}
   173  	}
   174  	return response, nil
   175  }