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

     1  package kafka
     2  
     3  import (
     4  	"context"
     5  	"errors"
     6  	"fmt"
     7  	"net"
     8  
     9  	"github.com/segmentio/kafka-go/protocol"
    10  	produceAPI "github.com/segmentio/kafka-go/protocol/produce"
    11  	"github.com/segmentio/kafka-go/protocol/rawproduce"
    12  )
    13  
    14  // RawProduceRequest represents a request sent to a kafka broker to produce records
    15  // to a topic partition. The request contains a pre-encoded/raw record set.
    16  type RawProduceRequest struct {
    17  	// Address of the kafka broker to send the request to.
    18  	Addr net.Addr
    19  
    20  	// The topic to produce the records to.
    21  	Topic string
    22  
    23  	// The partition to produce the records to.
    24  	Partition int
    25  
    26  	// The level of required acknowledgements to ask the kafka broker for.
    27  	RequiredAcks RequiredAcks
    28  
    29  	// The message format version used when encoding the records.
    30  	//
    31  	// By default, the client automatically determine which version should be
    32  	// used based on the version of the Produce API supported by the server.
    33  	MessageVersion int
    34  
    35  	// An optional transaction id when producing to the kafka broker is part of
    36  	// a transaction.
    37  	TransactionalID string
    38  
    39  	// The sequence of records to produce to the topic partition.
    40  	RawRecords protocol.RawRecordSet
    41  }
    42  
    43  // RawProduce sends a raw produce request to a kafka broker and returns the response.
    44  //
    45  // If the request contained no records, an error wrapping protocol.ErrNoRecord
    46  // is returned.
    47  //
    48  // When the request is configured with RequiredAcks=none, both the response and
    49  // the error will be nil on success.
    50  func (c *Client) RawProduce(ctx context.Context, req *RawProduceRequest) (*ProduceResponse, error) {
    51  	m, err := c.roundTrip(ctx, req.Addr, &rawproduce.Request{
    52  		TransactionalID: req.TransactionalID,
    53  		Acks:            int16(req.RequiredAcks),
    54  		Timeout:         c.timeoutMs(ctx, defaultProduceTimeout),
    55  		Topics: []rawproduce.RequestTopic{{
    56  			Topic: req.Topic,
    57  			Partitions: []rawproduce.RequestPartition{{
    58  				Partition: int32(req.Partition),
    59  				RecordSet: req.RawRecords,
    60  			}},
    61  		}},
    62  	})
    63  
    64  	switch {
    65  	case err == nil:
    66  	case errors.Is(err, protocol.ErrNoRecord):
    67  		return new(ProduceResponse), nil
    68  	default:
    69  		return nil, fmt.Errorf("kafka.(*Client).RawProduce: %w", err)
    70  	}
    71  
    72  	if req.RequiredAcks == RequireNone {
    73  		return nil, nil
    74  	}
    75  
    76  	res := m.(*produceAPI.Response)
    77  	if len(res.Topics) == 0 {
    78  		return nil, fmt.Errorf("kafka.(*Client).RawProduce: %w", protocol.ErrNoTopic)
    79  	}
    80  	topic := &res.Topics[0]
    81  	if len(topic.Partitions) == 0 {
    82  		return nil, fmt.Errorf("kafka.(*Client).RawProduce: %w", protocol.ErrNoPartition)
    83  	}
    84  	partition := &topic.Partitions[0]
    85  
    86  	ret := &ProduceResponse{
    87  		Throttle:       makeDuration(res.ThrottleTimeMs),
    88  		Error:          makeError(partition.ErrorCode, partition.ErrorMessage),
    89  		BaseOffset:     partition.BaseOffset,
    90  		LogAppendTime:  makeTime(partition.LogAppendTime),
    91  		LogStartOffset: partition.LogStartOffset,
    92  	}
    93  
    94  	if len(partition.RecordErrors) != 0 {
    95  		ret.RecordErrors = make(map[int]error, len(partition.RecordErrors))
    96  
    97  		for _, recErr := range partition.RecordErrors {
    98  			ret.RecordErrors[int(recErr.BatchIndex)] = errors.New(recErr.BatchIndexErrorMessage)
    99  		}
   100  	}
   101  
   102  	return ret, nil
   103  }