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

     1  package rawproduce
     2  
     3  import (
     4  	"fmt"
     5  
     6  	"github.com/segmentio/kafka-go/protocol"
     7  	"github.com/segmentio/kafka-go/protocol/produce"
     8  )
     9  
    10  func init() {
    11  	// Register a type override so that raw produce requests will be encoded with the correct type.
    12  	req := &Request{}
    13  	protocol.RegisterOverride(req, &produce.Response{}, req.TypeKey())
    14  }
    15  
    16  type Request struct {
    17  	TransactionalID string         `kafka:"min=v3,max=v8,nullable"`
    18  	Acks            int16          `kafka:"min=v0,max=v8"`
    19  	Timeout         int32          `kafka:"min=v0,max=v8"`
    20  	Topics          []RequestTopic `kafka:"min=v0,max=v8"`
    21  }
    22  
    23  func (r *Request) ApiKey() protocol.ApiKey { return protocol.Produce }
    24  
    25  func (r *Request) TypeKey() protocol.OverrideTypeKey { return protocol.RawProduceOverride }
    26  
    27  func (r *Request) Broker(cluster protocol.Cluster) (protocol.Broker, error) {
    28  	broker := protocol.Broker{ID: -1}
    29  
    30  	for i := range r.Topics {
    31  		t := &r.Topics[i]
    32  
    33  		topic, ok := cluster.Topics[t.Topic]
    34  		if !ok {
    35  			return broker, NewError(protocol.NewErrNoTopic(t.Topic))
    36  		}
    37  
    38  		for j := range t.Partitions {
    39  			p := &t.Partitions[j]
    40  
    41  			partition, ok := topic.Partitions[p.Partition]
    42  			if !ok {
    43  				return broker, NewError(protocol.NewErrNoPartition(t.Topic, p.Partition))
    44  			}
    45  
    46  			if b, ok := cluster.Brokers[partition.Leader]; !ok {
    47  				return broker, NewError(protocol.NewErrNoLeader(t.Topic, p.Partition))
    48  			} else if broker.ID < 0 {
    49  				broker = b
    50  			} else if b.ID != broker.ID {
    51  				return broker, NewError(fmt.Errorf("mismatching leaders (%d!=%d)", b.ID, broker.ID))
    52  			}
    53  		}
    54  	}
    55  
    56  	return broker, nil
    57  }
    58  
    59  func (r *Request) HasResponse() bool {
    60  	return r.Acks != 0
    61  }
    62  
    63  type RequestTopic struct {
    64  	Topic      string             `kafka:"min=v0,max=v8"`
    65  	Partitions []RequestPartition `kafka:"min=v0,max=v8"`
    66  }
    67  
    68  type RequestPartition struct {
    69  	Partition int32                 `kafka:"min=v0,max=v8"`
    70  	RecordSet protocol.RawRecordSet `kafka:"min=v0,max=v8"`
    71  }
    72  
    73  var (
    74  	_ protocol.BrokerMessage = (*Request)(nil)
    75  )
    76  
    77  type Error struct {
    78  	Err error
    79  }
    80  
    81  func NewError(err error) *Error {
    82  	return &Error{Err: err}
    83  }
    84  
    85  func (e *Error) Error() string {
    86  	return fmt.Sprintf("fetch request error: %v", e.Err)
    87  }
    88  
    89  func (e *Error) Unwrap() error {
    90  	return e.Err
    91  }