github.com/rbisecke/kafka-go@v0.4.27/protocol/describeconfigs/describeconfigs.go (about)

     1  package describeconfigs
     2  
     3  import (
     4  	"strconv"
     5  
     6  	"github.com/rbisecke/kafka-go/protocol"
     7  )
     8  
     9  const (
    10  	resourceTypeBroker int8 = 4
    11  )
    12  
    13  func init() {
    14  	protocol.Register(&Request{}, &Response{})
    15  }
    16  
    17  // Detailed API definition: https://kafka.apache.org/protocol#The_Messages_DescribeConfigs
    18  type Request struct {
    19  	Resources            []RequestResource `kafka:"min=v0,max=v3"`
    20  	IncludeSynonyms      bool              `kafka:"min=v1,max=v3"`
    21  	IncludeDocumentation bool              `kafka:"min=v3,max=v3"`
    22  }
    23  
    24  func (r *Request) ApiKey() protocol.ApiKey { return protocol.DescribeConfigs }
    25  
    26  func (r *Request) Broker(cluster protocol.Cluster) (protocol.Broker, error) {
    27  	// Broker metadata requests must be sent to the associated broker
    28  	for _, resource := range r.Resources {
    29  		if resource.ResourceType == resourceTypeBroker {
    30  			brokerID, err := strconv.Atoi(resource.ResourceName)
    31  			if err != nil {
    32  				return protocol.Broker{}, err
    33  			}
    34  
    35  			return cluster.Brokers[int32(brokerID)], nil
    36  		}
    37  	}
    38  
    39  	return cluster.Brokers[cluster.Controller], nil
    40  }
    41  
    42  func (r *Request) Split(cluster protocol.Cluster) (
    43  	[]protocol.Message,
    44  	protocol.Merger,
    45  	error,
    46  ) {
    47  	messages := []protocol.Message{}
    48  	topicsMessage := Request{}
    49  
    50  	for _, resource := range r.Resources {
    51  		// Split out broker requests to separate brokers
    52  		if resource.ResourceType == resourceTypeBroker {
    53  			messages = append(messages, &Request{
    54  				Resources: []RequestResource{resource},
    55  			})
    56  		} else {
    57  			topicsMessage.Resources = append(
    58  				topicsMessage.Resources, resource,
    59  			)
    60  		}
    61  	}
    62  
    63  	if len(topicsMessage.Resources) > 0 {
    64  		messages = append(messages, &topicsMessage)
    65  	}
    66  
    67  	return messages, new(Response), nil
    68  }
    69  
    70  type RequestResource struct {
    71  	ResourceType int8     `kafka:"min=v0,max=v3"`
    72  	ResourceName string   `kafka:"min=v0,max=v3"`
    73  	ConfigNames  []string `kafka:"min=v0,max=v3,nullable"`
    74  }
    75  
    76  type Response struct {
    77  	ThrottleTimeMs int32              `kafka:"min=v0,max=v3"`
    78  	Resources      []ResponseResource `kafka:"min=v0,max=v3"`
    79  }
    80  
    81  func (r *Response) ApiKey() protocol.ApiKey { return protocol.DescribeConfigs }
    82  
    83  func (r *Response) Merge(requests []protocol.Message, results []interface{}) (
    84  	protocol.Message,
    85  	error,
    86  ) {
    87  	response := &Response{}
    88  
    89  	for _, result := range results {
    90  		brokerResp := result.(*Response)
    91  		response.Resources = append(
    92  			response.Resources,
    93  			brokerResp.Resources...,
    94  		)
    95  	}
    96  
    97  	return response, nil
    98  }
    99  
   100  type ResponseResource struct {
   101  	ErrorCode     int16                 `kafka:"min=v0,max=v3"`
   102  	ErrorMessage  string                `kafka:"min=v0,max=v3,nullable"`
   103  	ResourceType  int8                  `kafka:"min=v0,max=v3"`
   104  	ResourceName  string                `kafka:"min=v0,max=v3"`
   105  	ConfigEntries []ResponseConfigEntry `kafka:"min=v0,max=v3"`
   106  }
   107  
   108  type ResponseConfigEntry struct {
   109  	ConfigName          string                  `kafka:"min=v0,max=v3"`
   110  	ConfigValue         string                  `kafka:"min=v0,max=v3,nullable"`
   111  	ReadOnly            bool                    `kafka:"min=v0,max=v3"`
   112  	IsDefault           bool                    `kafka:"min=v0,max=v0"`
   113  	ConfigSource        int8                    `kafka:"min=v1,max=v3"`
   114  	IsSensitive         bool                    `kafka:"min=v0,max=v3"`
   115  	ConfigSynonyms      []ResponseConfigSynonym `kafka:"min=v1,max=v3"`
   116  	ConfigType          int8                    `kafka:"min=v3,max=v3"`
   117  	ConfigDocumentation string                  `kafka:"min=v3,max=v3,nullable"`
   118  }
   119  
   120  type ResponseConfigSynonym struct {
   121  	ConfigName   string `kafka:"min=v1,max=v3"`
   122  	ConfigValue  string `kafka:"min=v1,max=v3,nullable"`
   123  	ConfigSource int8   `kafka:"min=v1,max=v3"`
   124  }
   125  
   126  var (
   127  	_ protocol.BrokerMessage = (*Request)(nil)
   128  )