github.com/deanMdreon/kafka-go@v0.4.32/protocol/createtopics/createtopics.go (about)

     1  package createtopics
     2  
     3  import "github.com/deanMdreon/kafka-go/protocol"
     4  
     5  func init() {
     6  	protocol.Register(&Request{}, &Response{})
     7  }
     8  
     9  type Request struct {
    10  	// We need at least one tagged field to indicate that v5+ uses "flexible"
    11  	// messages.
    12  	_ struct{} `kafka:"min=v5,max=v5,tag"`
    13  
    14  	Topics       []RequestTopic `kafka:"min=v0,max=v5"`
    15  	TimeoutMs    int32          `kafka:"min=v0,max=v5"`
    16  	ValidateOnly bool           `kafka:"min=v1,max=v5"`
    17  }
    18  
    19  func (r *Request) ApiKey() protocol.ApiKey { return protocol.CreateTopics }
    20  
    21  func (r *Request) Broker(cluster protocol.Cluster) (protocol.Broker, error) {
    22  	return cluster.Brokers[cluster.Controller], nil
    23  }
    24  
    25  type RequestTopic struct {
    26  	Name              string              `kafka:"min=v0,max=v5"`
    27  	NumPartitions     int32               `kafka:"min=v0,max=v5"`
    28  	ReplicationFactor int16               `kafka:"min=v0,max=v5"`
    29  	Assignments       []RequestAssignment `kafka:"min=v0,max=v5"`
    30  	Configs           []RequestConfig     `kafka:"min=v0,max=v5"`
    31  }
    32  
    33  type RequestAssignment struct {
    34  	PartitionIndex int32   `kafka:"min=v0,max=v5"`
    35  	BrokerIDs      []int32 `kafka:"min=v0,max=v5"`
    36  }
    37  
    38  type RequestConfig struct {
    39  	Name  string `kafka:"min=v0,max=v5"`
    40  	Value string `kafka:"min=v0,max=v5,nullable"`
    41  }
    42  
    43  type Response struct {
    44  	// We need at least one tagged field to indicate that v5+ uses "flexible"
    45  	// messages.
    46  	_ struct{} `kafka:"min=v5,max=v5,tag"`
    47  
    48  	ThrottleTimeMs int32           `kafka:"min=v2,max=v5"`
    49  	Topics         []ResponseTopic `kafka:"min=v0,max=v5"`
    50  }
    51  
    52  func (r *Response) ApiKey() protocol.ApiKey { return protocol.CreateTopics }
    53  
    54  type ResponseTopic struct {
    55  	Name              string `kafka:"min=v0,max=v5"`
    56  	ErrorCode         int16  `kafka:"min=v0,max=v5"`
    57  	ErrorMessage      string `kafka:"min=v1,max=v5,nullable"`
    58  	NumPartitions     int32  `kafka:"min=v5,max=v5"`
    59  	ReplicationFactor int16  `kafka:"min=v5,max=v5"`
    60  
    61  	Configs []ResponseTopicConfig `kafka:"min=v5,max=v5"`
    62  }
    63  
    64  type ResponseTopicConfig struct {
    65  	Name         string `kafka:"min=v5,max=v5"`
    66  	Value        string `kafka:"min=v5,max=v5,nullable"`
    67  	ReadOnly     bool   `kafka:"min=v5,max=v5"`
    68  	ConfigSource int8   `kafka:"min=v5,max=v5"`
    69  	IsSensitive  bool   `kafka:"min=v5,max=v5"`
    70  }
    71  
    72  var (
    73  	_ protocol.BrokerMessage = (*Request)(nil)
    74  )