diff --git a/CHANGELOG.md b/CHANGELOG.md index 4c5d31d95..50f4d89cd 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -15,6 +15,7 @@ alert will auto-recover. This is because the new v2 API is not structured with static URLs, and so only the action can be defined and not the entire URL. - [#1690](https://github.com/influxdata/kapacitor/issues/1690): Add https-private-key option to httpd config. - [#1561](https://github.com/influxdata/kapacitor/issues/1561): Add .quiet to all nodes to silence any errors reported by the node. +- [#1826](https://github.com/influxdata/kapacitor/issues/1826): Add Kafka alert handler. ### Bugfixes - [#1794](https://github.com/influxdata/kapacitor/issues/1794): Kapacitor ticks generating a hash instead of their actual given name. diff --git a/Gopkg.lock b/Gopkg.lock index 144227c28..b035c30ff 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -570,6 +570,12 @@ packages = ["zk"] revision = "1d7be4effb13d2d908342d349d71a284a7542693" +[[projects]] + branch = "master" + name = "github.com/segmentio/kafka-go" + packages = ["."] + revision = "140b1a903e1421f33b16a24312a19ff024e13625" + [[projects]] branch = "master" name = "github.com/serenize/snaker" @@ -886,6 +892,6 @@ [solve-meta] analyzer-name = "dep" analyzer-version = 1 - inputs-digest = "553955f0d04a6196beef13bce4c1117cbfe5c9c6b1f08cc4337443eb7d24b292" + inputs-digest = "471285daf6e6beceaa938816b65b7b835f7e8883968f6c7582f830173bb57a73" solver-name = "gps-cdcl" solver-version = 1 diff --git a/LICENSE_OF_DEPENDENCIES.md b/LICENSE_OF_DEPENDENCIES.md index 9c7d6a2ea..81699f63b 100644 --- a/LICENSE_OF_DEPENDENCIES.md +++ b/LICENSE_OF_DEPENDENCIES.md @@ -16,6 +16,7 @@ Dependencies * github.com/mitchellh/reflectwalk [MIT](https://github.com/mitchellh/reflectwalk/blob/master/LICENSE) * github.com/pkg/errors [BSD](https://github.com/pkg/errors/blob/master/LICENSE) * github.com/russross/blackfriday [BSD](https://github.com/russross/blackfriday/blob/master/LICENSE.txt) +* github.com/segmentio/kafka-go [MIT](https://github.com/segmentio/kafka-go/blob/master/LICENSE) * github.com/serenize/snaker [MIT](https://github.com/serenize/snaker/blob/master/LICENSE.txt) * github.com/shurcooL/go [MIT](https://github.com/shurcooL/go/blob/master/README.md) * github.com/shurcooL/markdownfmt [MIT](https://github.com/shurcooL/markdownfmt/blob/master/README.md) diff --git a/alert.go b/alert.go index 0a0a4091d..4dcc9b5fe 100644 --- a/alert.go +++ b/alert.go @@ -19,6 +19,7 @@ import ( alertservice "github.com/influxdata/kapacitor/services/alert" "github.com/influxdata/kapacitor/services/hipchat" "github.com/influxdata/kapacitor/services/httppost" + "github.com/influxdata/kapacitor/services/kafka" "github.com/influxdata/kapacitor/services/mqtt" "github.com/influxdata/kapacitor/services/opsgenie" "github.com/influxdata/kapacitor/services/opsgenie2" @@ -315,6 +316,19 @@ func newAlertNode(et *ExecutingTask, n *pipeline.AlertNode, d NodeDiagnostic) (a n.IsStateChangesOnly = true } + for _, k := range n.KafkaHandlers { + c := kafka.HandlerConfig{ + Cluster: k.Cluster, + Topic: k.Topic, + Template: k.Template, + } + h, err := et.tm.KafkaService.Handler(c, ctx...) + if err != nil { + return nil, errors.Wrapf(err, "failed to create kafka handler") + } + an.handlers = append(an.handlers, h) + } + for _, a := range n.AlertaHandlers { c := et.tm.AlertaService.DefaultHandlerConfig() if a.Token != "" { diff --git a/etc/kapacitor/kapacitor.conf b/etc/kapacitor/kapacitor.conf index 926afbbae..ff55f26e1 100644 --- a/etc/kapacitor/kapacitor.conf +++ b/etc/kapacitor/kapacitor.conf @@ -448,6 +448,34 @@ default-retention-policy = "" # meaning alerts will only be sent if the alert state changes. state-changes-only = false +[[kafka]] + # Configure Kafka + enabled = false + # ID is a unique identifier for this kafka cluster. + id = "localhost" + # Brokers is a list of host:port addresses of Kafka brokers. + brokers = [] + # Timeout on network operations with the brokers. + # If 0 a default of 10s will be used. + timeout = 10s + # BatchSize is the number of messages that are batched before being sent to Kafka + # If 0 a default of 100 will be used. + batch-size = 100 + # BatchTimeout is the maximum amount of time to wait before flushing an incomplete batch. + # If 0 a default of 1s will be used. + batch-timeout = 1s + # Use SSL enables ssl communication. + # Must be true for the other ssl options to take effect. + use-ssl = false + # Path to CA file + ssl-ca = "" + # Path to host cert file + ssl-cert = "" + # Path to cert key file + ssl-key = "" + # Use SSL but skip chain & host verification + insecure-skip-verify = false + [alerta] # Configure Alerta. enabled = false diff --git a/pipeline/alert.go b/pipeline/alert.go index 369c6f8fd..ceb2333a9 100644 --- a/pipeline/alert.go +++ b/pipeline/alert.go @@ -381,6 +381,10 @@ type AlertNodeData struct { // Send alert using SNMPtraps. // tick:ignore SNMPTrapHandlers []*SNMPTrapHandler `tick:"SnmpTrap" json:"snmpTrap"` + + // Send alert to Kafka topic + // tick:ignore + KafkaHandlers []*KafkaHandler `tick:"Kafka" json:"kafka"` } func newAlertNode(wants EdgeType) *AlertNode { @@ -1833,3 +1837,44 @@ func (h *SNMPTrapHandler) validate() error { } return nil } + +// Send the alert to a Kafka topic. +// +// Example: +// [[kafka]] +// enabled = true +// id = "default" +// brokers = ["localhost:9092"] +// +// Example: +// stream +// |alert() +// .kafka() +// .cluster('default') +// .kafkaTopic('alerts') +// +// +// tick:property +func (n *AlertNodeData) Kafka() *KafkaHandler { + k := &KafkaHandler{ + AlertNodeData: n, + } + n.KafkaHandlers = append(n.KafkaHandlers, k) + return k +} + +// Kafka alert Handler +// tick:embedded:AlertNode.Kafka +type KafkaHandler struct { + *AlertNodeData `json:"-"` + + // Cluster is the id of the configure kafka cluster + Cluster string `json:"cluster"` + + // Kafka Topic + KafkaTopic string `json:"kafka-topic"` + + // Template used to construct the message body + // If empty the alert data in JSON is sent as the message body. + Template string `json:"template"` +} diff --git a/pipeline/alert_test.go b/pipeline/alert_test.go index 1ca7d41dc..3a2eaecdc 100644 --- a/pipeline/alert_test.go +++ b/pipeline/alert_test.go @@ -78,7 +78,8 @@ func TestAlertNode_MarshalJSON(t *testing.T) { "opsGenie2": null, "talk": null, "mqtt": null, - "snmpTrap": null + "snmpTrap": null, + "kafka": null }`, }, } diff --git a/pipeline/json_test.go b/pipeline/json_test.go index 04f2740d6..6c9b5227d 100644 --- a/pipeline/json_test.go +++ b/pipeline/json_test.go @@ -257,7 +257,8 @@ func TestPipeline_MarshalJSON(t *testing.T) { "opsGenie2": null, "talk": null, "mqtt": null, - "snmpTrap": null + "snmpTrap": null, + "kafka": null }, { "typeOf": "httpOut", diff --git a/pipeline/tick/alert.go b/pipeline/tick/alert.go index b922f193f..6786eb8bc 100644 --- a/pipeline/tick/alert.go +++ b/pipeline/tick/alert.go @@ -161,6 +161,13 @@ func (n *AlertNode) Build(a *pipeline.AlertNode) (ast.Node, error) { Dot("token", h.Token) } + for _, h := range a.KafkaHandlers { + n.Dot("kafka"). + Dot("cluster", h.Cluster). + Dot("kafkaTopic", h.KafkaTopic). + Dot("template", h.Template) + } + for _, h := range a.AlertaHandlers { n.Dot("alerta"). Dot("token", h.Token). diff --git a/pipeline/tick/alert_test.go b/pipeline/tick/alert_test.go index e7e2f6fe1..e5f3f7e00 100644 --- a/pipeline/tick/alert_test.go +++ b/pipeline/tick/alert_test.go @@ -443,6 +443,28 @@ func TestAlertHipchat(t *testing.T) { PipelineTickTestHelper(t, pipe, want) } +func TestAlertKafka(t *testing.T) { + pipe, _, from := StreamFrom() + handler := from.Alert().Kafka() + handler.Cluster = "default" + handler.KafkaTopic = "test" + handler.Template = "tmpl" + + want := `stream + |from() + |alert() + .id('{{ .Name }}:{{ .Group }}') + .message('{{ .ID }} is {{ .Level }}') + .details('{{ json . }}') + .history(21) + .kafka() + .cluster('default') + .kafkaTopic('test') + .template('tmpl') +` + PipelineTickTestHelper(t, pipe, want) +} + func TestAlertAlerta(t *testing.T) { pipe, _, from := StreamFrom() handler := from.Alert().Alerta() diff --git a/server/config.go b/server/config.go index fab54f543..4ec4d3fc9 100644 --- a/server/config.go +++ b/server/config.go @@ -27,6 +27,7 @@ import ( "github.com/influxdata/kapacitor/services/httppost" "github.com/influxdata/kapacitor/services/influxdb" "github.com/influxdata/kapacitor/services/k8s" + "github.com/influxdata/kapacitor/services/kafka" "github.com/influxdata/kapacitor/services/load" "github.com/influxdata/kapacitor/services/marathon" "github.com/influxdata/kapacitor/services/mqtt" @@ -82,6 +83,7 @@ type Config struct { // Alert handlers Alerta alerta.Config `toml:"alerta" override:"alerta"` HipChat hipchat.Config `toml:"hipchat" override:"hipchat"` + Kafka kafka.Configs `toml:"kafka" override:"kafka,element-key=id"` MQTT mqtt.Configs `toml:"mqtt" override:"mqtt,element-key=name"` OpsGenie opsgenie.Config `toml:"opsgenie" override:"opsgenie"` OpsGenie2 opsgenie2.Config `toml:"opsgenie2" override:"opsgenie2"` @@ -263,6 +265,9 @@ func (c *Config) Validate() error { if err := c.HipChat.Validate(); err != nil { return errors.Wrap(err, "hipchat") } + if err := c.Kafka.Validate(); err != nil { + return errors.Wrap(err, "kafka") + } if err := c.MQTT.Validate(); err != nil { return errors.Wrap(err, "mqtt") } diff --git a/server/server.go b/server/server.go index 1233ee48f..8b159d609 100644 --- a/server/server.go +++ b/server/server.go @@ -37,6 +37,7 @@ import ( "github.com/influxdata/kapacitor/services/httppost" "github.com/influxdata/kapacitor/services/influxdb" "github.com/influxdata/kapacitor/services/k8s" + "github.com/influxdata/kapacitor/services/kafka" "github.com/influxdata/kapacitor/services/load" "github.com/influxdata/kapacitor/services/marathon" "github.com/influxdata/kapacitor/services/mqtt" @@ -229,6 +230,7 @@ func New(c *Config, buildInfo BuildInfo, diagService *diagnostic.Service) (*Serv // Append Alert integration services s.appendAlertaService() s.appendHipChatService() + s.appendKafkaService() if err := s.appendMQTTService(); err != nil { return nil, errors.Wrap(err, "mqtt service") } @@ -731,6 +733,18 @@ func (s *Server) appendHipChatService() { s.AppendService("hipchat", srv) } +func (s *Server) appendKafkaService() { + c := s.config.Kafka + d := s.DiagService.NewKafkaHandler() + srv := kafka.NewService(c, d) + + s.TaskMaster.KafkaService = srv + s.AlertService.KafkaService = srv + + s.SetDynamicService("kafka", srv) + s.AppendService("kafka", srv) +} + func (s *Server) appendAlertaService() { c := s.config.Alerta d := s.DiagService.NewAlertaHandler() diff --git a/server/server_test.go b/server/server_test.go index f720de12a..a4c737c50 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -41,6 +41,8 @@ import ( "github.com/influxdata/kapacitor/services/httppost" "github.com/influxdata/kapacitor/services/httppost/httpposttest" "github.com/influxdata/kapacitor/services/k8s" + "github.com/influxdata/kapacitor/services/kafka" + "github.com/influxdata/kapacitor/services/kafka/kafkatest" "github.com/influxdata/kapacitor/services/mqtt" "github.com/influxdata/kapacitor/services/mqtt/mqtttest" "github.com/influxdata/kapacitor/services/opsgenie" @@ -8699,6 +8701,16 @@ func TestServer_ListServiceTests(t *testing.T) { "cluster": "", }, }, + { + Link: client.Link{Relation: client.Self, Href: "/kapacitor/v1/service-tests/kafka"}, + Name: "kafka", + Options: client.ServiceTestOptions{ + "cluster": "example", + "topic": "test", + "key": "key", + "message": "test kafka message", + }, + }, { Link: client.Link{Relation: client.Self, Href: "/kapacitor/v1/service-tests/kubernetes"}, Name: "kubernetes", @@ -9505,6 +9517,49 @@ func TestServer_AlertHandlers(t *testing.T) { return nil }, }, + { + handler: client.TopicHandler{ + Kind: "kafka", + Options: map[string]interface{}{ + "cluster": "default", + "topic": "test", + }, + }, + setup: func(c *server.Config, ha *client.TopicHandler) (context.Context, error) { + ts, err := kafkatest.NewServer() + if err != nil { + return nil, err + } + ctxt := context.WithValue(nil, "server", ts) + + c.Kafka = kafka.Configs{{ + Enabled: true, + ID: "default", + Brokers: []string{ts.Addr.String()}, + }} + return ctxt, nil + }, + result: func(ctxt context.Context) error { + ts := ctxt.Value("server").(*kafkatest.Server) + time.Sleep(2 * time.Second) + ts.Close() + got, err := ts.Messages() + if err != nil { + return err + } + exp := []kafkatest.Message{{ + Topic: "test", + Partition: 1, + Offset: 0, + Key: "id", + Message: string(adJSON) + "\n", + }} + if !cmp.Equal(exp, got) { + return fmt.Errorf("unexpected kafak messages -exp/+got:\n%s", cmp.Diff(exp, got)) + } + return nil + }, + }, { handler: client.TopicHandler{ Kind: "log", diff --git a/services/alert/service.go b/services/alert/service.go index be821f253..2650d969c 100644 --- a/services/alert/service.go +++ b/services/alert/service.go @@ -17,6 +17,7 @@ import ( "github.com/influxdata/kapacitor/services/hipchat" "github.com/influxdata/kapacitor/services/httpd" "github.com/influxdata/kapacitor/services/httppost" + "github.com/influxdata/kapacitor/services/kafka" "github.com/influxdata/kapacitor/services/mqtt" "github.com/influxdata/kapacitor/services/opsgenie" "github.com/influxdata/kapacitor/services/opsgenie2" @@ -85,6 +86,9 @@ type Service struct { HipChatService interface { Handler(hipchat.HandlerConfig, ...keyvalue.T) alert.Handler } + KafkaService interface { + Handler(kafka.HandlerConfig, ...keyvalue.T) (alert.Handler, error) + } MQTTService interface { Handler(mqtt.HandlerConfig, ...keyvalue.T) alert.Handler } @@ -787,6 +791,17 @@ func (s *Service) createHandlerFromSpec(spec HandlerSpec) (handler, error) { } h = s.HipChatService.Handler(c, ctx...) h = newExternalHandler(h) + case "kafka": + c := kafka.HandlerConfig{} + err = decodeOptions(spec.Options, &c) + if err != nil { + return handler{}, err + } + h, err = s.KafkaService.Handler(c, ctx...) + if err != nil { + return handler{}, err + } + h = newExternalHandler(h) case "log": c := DefaultLogHandlerConfig() err = decodeOptions(spec.Options, &c) diff --git a/services/diagnostic/handlers.go b/services/diagnostic/handlers.go index 79f5d27b8..06dce758a 100644 --- a/services/diagnostic/handlers.go +++ b/services/diagnostic/handlers.go @@ -22,6 +22,7 @@ import ( "github.com/influxdata/kapacitor/services/httppost" "github.com/influxdata/kapacitor/services/influxdb" "github.com/influxdata/kapacitor/services/k8s" + "github.com/influxdata/kapacitor/services/kafka" "github.com/influxdata/kapacitor/services/mqtt" "github.com/influxdata/kapacitor/services/opsgenie" "github.com/influxdata/kapacitor/services/opsgenie2" @@ -409,6 +410,26 @@ func (h *HipChatHandler) Error(msg string, err error) { h.l.Error(msg, Error(err)) } +// Kafka handler +type KafkaHandler struct { + l Logger +} + +func (h *KafkaHandler) WithContext(ctx ...keyvalue.T) kafka.Diagnostic { + fields := logFieldsFromContext(ctx) + + return &KafkaHandler{ + l: h.l.With(fields...), + } +} + +func (h *KafkaHandler) Error(msg string, err error) { + h.l.Error(msg, Error(err)) +} +func (h *KafkaHandler) InsecureSkipVerify() { + h.l.Info("service is configured to skip ssl verification") +} + // HTTPD handler type HTTPDHandler struct { diff --git a/services/diagnostic/service.go b/services/diagnostic/service.go index 3cb590131..750de1566 100644 --- a/services/diagnostic/service.go +++ b/services/diagnostic/service.go @@ -196,6 +196,12 @@ func (s *Service) NewHipChatHandler() *HipChatHandler { } } +func (s *Service) NewKafkaHandler() *KafkaHandler { + return &KafkaHandler{ + l: s.Logger.With(String("service", "kafka")), + } +} + func (s *Service) NewPagerDutyHandler() *PagerDutyHandler { return &PagerDutyHandler{ l: s.Logger.With(String("service", "pagerduty")), diff --git a/services/kafka/config.go b/services/kafka/config.go new file mode 100644 index 000000000..8b38aa68a --- /dev/null +++ b/services/kafka/config.go @@ -0,0 +1,105 @@ +package kafka + +import ( + "crypto/tls" + "time" + + "github.com/influxdata/influxdb/toml" + "github.com/influxdata/kapacitor/tlsconfig" + "github.com/pkg/errors" + kafka "github.com/segmentio/kafka-go" +) + +const ( + DefaultTimeout = 10 * time.Second + DefaultBatchSize = 100 + DefaultBatchTimeout = 1 * time.Second +) + +type Config struct { + Enabled bool `toml:"enabled" override:"enabled"` + // ID is a unique identifier for this Kafka config + ID string `toml:"id" override:"id"` + // Brokers is a list of host:port addresses of Kafka brokers. + Brokers []string `toml:"brokers" override:"brokers"` + // Timeout on network operations with the brokers. + // If 0 a default of 10s will be used. + Timeout toml.Duration `toml:"timeout" override:"timeout"` + // BatchSize is the number of messages that are batched before being sent to Kafka + // If 0 a default of 100 will be used. + BatchSize int `toml:"batch-size" override:"batch-size"` + // BatchTimeout is the maximum amount of time to wait before flushing an incomplete batch. + // If 0 a default of 1s will be used. + BatchTimeout toml.Duration `toml:"batch-timeout" override:"batch-timeout"` + // UseSSL enable ssl communication + // Must be true for the other ssl options to take effect. + UseSSL bool `toml:"use-ssl" override:"use-ssl"` + // Path to CA file + SSLCA string `toml:"ssl-ca" override:"ssl-ca"` + // Path to host cert file + SSLCert string `toml:"ssl-cert" override:"ssl-cert"` + // Path to cert key file + SSLKey string `toml:"ssl-key" override:"ssl-key"` + // Use SSL but skip chain & host verification + InsecureSkipVerify bool `toml:"insecure-skip-verify" override:"insecure-skip-verify"` +} + +func (c Config) Validate() error { + if !c.Enabled { + return nil + } + // ID must not be empty + if c.ID == "" { + return errors.New("id must not be empty") + } + if len(c.Brokers) == 0 { + return errors.New("no brokers specified, must provide at least one broker URL") + } + return nil +} + +func (c *Config) ApplyConditionalDefaults() { + if c.Timeout == 0 { + c.Timeout = toml.Duration(DefaultTimeout) + } + if c.BatchSize == 0 { + c.BatchSize = DefaultBatchSize + } + if c.BatchTimeout == 0 { + c.BatchTimeout = toml.Duration(DefaultBatchTimeout) + } +} + +func (c Config) WriterConfig() (kafka.WriterConfig, error) { + var tlsCfg *tls.Config + if c.UseSSL { + t, err := tlsconfig.Create(c.SSLCA, c.SSLCert, c.SSLKey, c.InsecureSkipVerify) + if err != nil { + return kafka.WriterConfig{}, err + } + tlsCfg = t + } + dialer := &kafka.Dialer{ + Timeout: time.Duration(c.Timeout), + TLS: tlsCfg, + } + return kafka.WriterConfig{ + Brokers: c.Brokers, + Balancer: &kafka.LeastBytes{}, + Dialer: dialer, + ReadTimeout: time.Duration(c.Timeout), + WriteTimeout: time.Duration(c.Timeout), + BatchTimeout: time.Duration(c.BatchTimeout), + }, nil +} + +type Configs []Config + +func (cs Configs) Validate() error { + for _, c := range cs { + if err := c.Validate(); err != nil { + return err + } + } + return nil +} diff --git a/services/kafka/kafkatest/server.go b/services/kafka/kafkatest/server.go new file mode 100644 index 000000000..a52b2a585 --- /dev/null +++ b/services/kafka/kafkatest/server.go @@ -0,0 +1,329 @@ +package kafkatest + +import ( + "encoding/binary" + "fmt" + "io" + "net" + "strconv" + "strings" + "sync" +) + +// Provides an incomplete Kafka Server implementation. +// Records messages sent via the ProduceRequest and responds to MetadataRequests as the only broker. +type Server struct { + Addr net.Addr + mu sync.Mutex + wg sync.WaitGroup + + closed bool + closing chan struct{} + + messages []Message + errors []error + + brokerMessage []byte + nodeID int32 + + partitionMessage []byte +} + +func NewServer() (*Server, error) { + s := &Server{ + closing: make(chan struct{}), + nodeID: 1, + } + l, err := net.Listen("tcp", ":0") + if err != nil { + return nil, err + } + s.Addr = l.Addr() + + // Prepare static message bytes + s.prepareBrokerMsg() + s.preparePartitionMsg() + + // start server + s.wg.Add(1) + go func() { + defer s.wg.Done() + s.run(l) + }() + return s, nil +} + +func (s *Server) prepareBrokerMsg() { + host, port, _ := net.SplitHostPort(s.Addr.String()) + s.brokerMessage = make([]byte, 0, 4+2+len(host)+4) + s.brokerMessage = writeInt32(s.brokerMessage, s.nodeID) + s.brokerMessage = writeStr(s.brokerMessage, host) + portN, _ := strconv.Atoi(port) + s.brokerMessage = writeInt32(s.brokerMessage, int32(portN)) +} + +func (s *Server) preparePartitionMsg() { + s.partitionMessage = make([]byte, 0, 2+4+4+4+4) + // Write error code + s.partitionMessage = writeInt16(s.partitionMessage, 0) + // Write partition ID + s.partitionMessage = writeInt32(s.partitionMessage, 1) + // Write leader ID + s.partitionMessage = writeInt32(s.partitionMessage, s.nodeID) + // Write 0 len replicas + s.partitionMessage = writeArrayHeader(s.partitionMessage, 0) + // Write 0 len Isr + s.partitionMessage = writeArrayHeader(s.partitionMessage, 0) +} + +func (s *Server) Close() { + s.mu.Lock() + defer s.mu.Unlock() + if s.closed { + return + } + s.closed = true + close(s.closing) + s.wg.Wait() +} + +func (s *Server) Messages() ([]Message, error) { + if len(s.errors) > 0 { + return nil, multiError(s.errors) + } + return s.messages, nil +} + +func (s *Server) run(l net.Listener) { + defer l.Close() + + accepts := make(chan net.Conn) + + s.wg.Add(1) + go func() { + defer s.wg.Done() + for { + c, err := l.Accept() + if err != nil { + return + } + select { + case accepts <- c: + case <-s.closing: + return + } + } + }() + + for { + select { + case c := <-accepts: + s.wg.Add(1) + go func() { + defer s.wg.Done() + defer c.Close() + if err := s.handle(c); err != nil { + s.errors = append(s.errors, err) + } + }() + case <-s.closing: + return + } + } +} + +func (s *Server) handle(c net.Conn) error { + var size int32 + err := binary.Read(c, binary.BigEndian, &size) + if err != nil { + return err + } + buf := make([]byte, int(size)) + io.ReadFull(c, buf) + if err != nil { + return err + } + // ApiKey indicated the type of request + apiKey := int16(binary.BigEndian.Uint16(buf[:2])) + _, n := readStr(buf[8:]) + request := buf[8+n:] + + // Prepare response + response := make([]byte, 8, 1024) + // Leave first 4 bytes for the size. + // Copy correlationID. + copy(response[4:], buf[4:8]) + + switch apiKey { + case 0: // ProduceRequest + topic, partition, offset := s.readProduceRequest(request) + + // Prepare success response + response = writeArrayHeader(response, 1) + response = writeStr(response, topic) + response = writeArrayHeader(response, 1) + response = writeInt32(response, partition) + response = writeInt16(response, 0) // Error Code + response = writeInt64(response, offset) + response = writeInt64(response, -1) // Timestamp + response = writeInt32(response, 0) // ThrottleTime + case 3: // Metadata + topics, _ := readStrList(request) + + // Write broker message + response = writeArray(response, [][]byte{s.brokerMessage}) + + // Write topic metadata + response = writeArrayHeader(response, int32(len(topics))) + for _, t := range topics { + // Write Error Code + response = writeInt16(response, 0) + // Write topic name + response = writeStr(response, t) + + // Write partition + response = writeArray(response, [][]byte{s.partitionMessage}) + } + default: + return fmt.Errorf("unsupported apiKey %d", apiKey) + } + + // Set response size + responseSize := len(response) - 4 + binary.BigEndian.PutUint32(response[:4], uint32(responseSize)) + _, err = c.Write(response) + return err +} + +// readProduceRequest, assume only a single message exists +func (s *Server) readProduceRequest(request []byte) (topic string, partition int32, offset int64) { + pos := 2 + 4 + 4 // skip RequiredAcks and Timeout and array len + + // Read topic name + topic, n := readStr(request[pos:]) + pos += n + + pos += 4 // skip array len + + partition = readInt32(request[pos:]) + pos += 4 + + pos += 4 // skip set size + + offset = readInt64(request[pos:]) + pos += 8 + + pos += 4 + 4 + 1 + 1 + 8 // skip size, crc, magic, attributes, timestamp + + key, n := readByteArray(request[pos:]) + pos += n + + message, n := readByteArray(request[pos:]) + pos += n + + s.saveMessage(Message{ + Topic: topic, + Partition: partition, + Offset: offset, + Key: string(key), + Message: string(message), + }) + return +} + +func (s *Server) saveMessage(m Message) { + s.mu.Lock() + defer s.mu.Unlock() + s.messages = append(s.messages, m) +} + +func readList(buf []byte, f func([]byte) int) int { + pos := 4 + count := int(int32(binary.BigEndian.Uint32(buf[:pos]))) + for i := 0; i < count; i++ { + pos += f(buf[pos:]) + } + return pos +} +func readStrList(buf []byte) ([]string, int) { + var strs []string + l := readList(buf, func(data []byte) int { + s, n := readStr(data) + strs = append(strs, s) + return n + }) + return strs, l +} +func readStr(buf []byte) (string, int) { + n := int(int16(binary.BigEndian.Uint16(buf[:2]))) + return string(buf[2 : 2+n]), n + 2 +} +func readByteArray(buf []byte) ([]byte, int) { + n := int(int32(binary.BigEndian.Uint32(buf[:4]))) + return buf[4 : 4+n], n + 4 +} + +func readInt16(buf []byte) int16 { + return int16(binary.BigEndian.Uint16(buf[:2])) +} +func readInt32(buf []byte) int32 { + return int32(binary.BigEndian.Uint32(buf[:4])) +} +func readInt64(buf []byte) int64 { + return int64(binary.BigEndian.Uint64(buf[:8])) +} + +func writeStr(dst []byte, s string) []byte { + dst = writeInt16(dst, len(s)) + return append(dst, []byte(s)...) +} + +func writeInt16(dst []byte, n int) []byte { + l := len(dst) + dst = append(dst, []byte{0, 0}...) + binary.BigEndian.PutUint16(dst[l:l+2], uint16(n)) + return dst +} + +func writeInt32(dst []byte, n int32) []byte { + l := len(dst) + dst = append(dst, []byte{0, 0, 0, 0}...) + binary.BigEndian.PutUint32(dst[l:l+4], uint32(n)) + return dst +} +func writeInt64(dst []byte, n int64) []byte { + l := len(dst) + dst = append(dst, []byte{0, 0, 0, 0, 0, 0, 0, 0}...) + binary.BigEndian.PutUint64(dst[l:l+8], uint64(n)) + return dst +} + +func writeArrayHeader(dst []byte, n int32) []byte { + return writeInt32(dst, n) +} + +func writeArray(dst []byte, data [][]byte) []byte { + dst = writeArrayHeader(dst, int32(len(data))) + for _, d := range data { + dst = append(dst, d...) + } + return dst +} + +type multiError []error + +func (e multiError) Error() string { + errs := make([]string, len(e)) + for i := range e { + errs[i] = e[i].Error() + } + return strings.Join(errs, "\n") +} + +type Message struct { + Topic string + Partition int32 + Offset int64 + Key string + Message string +} diff --git a/services/kafka/service.go b/services/kafka/service.go new file mode 100644 index 000000000..837b39da6 --- /dev/null +++ b/services/kafka/service.go @@ -0,0 +1,283 @@ +package kafka + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "sort" + "sync" + "text/template" + + "github.com/influxdata/kapacitor/alert" + "github.com/influxdata/kapacitor/keyvalue" + "github.com/pkg/errors" + kafka "github.com/segmentio/kafka-go" +) + +type Diagnostic interface { + WithContext(ctx ...keyvalue.T) Diagnostic + InsecureSkipVerify() + Error(msg string, err error) +} + +type Cluster struct { + mu sync.RWMutex + cfg Config + + writers map[string]*kafka.Writer +} + +func NewCluster(c Config) *Cluster { + return &Cluster{ + cfg: c, + writers: make(map[string]*kafka.Writer), + } +} + +func (c *Cluster) WriteMessage(topic string, key, msg []byte) error { + w, err := c.writer(topic) + if err != nil { + return err + } + return w.WriteMessages(context.Background(), kafka.Message{ + Key: key, + Value: msg, + }) +} + +func (c *Cluster) writer(topic string) (*kafka.Writer, error) { + c.mu.RLock() + w, ok := c.writers[topic] + c.mu.RUnlock() + if !ok { + c.mu.Lock() + defer c.mu.Unlock() + w, ok = c.writers[topic] + if !ok { + wc, err := c.cfg.WriterConfig() + if err != nil { + return nil, err + } + wc.Topic = topic + w = kafka.NewWriter(wc) + c.writers[topic] = w + } + } + return w, nil +} + +func (c *Cluster) Close() { + c.mu.Lock() + defer c.mu.Unlock() + for _, w := range c.writers { + w.Close() + } + return +} + +func (c *Cluster) Update(cfg Config) error { + c.mu.Lock() + defer c.mu.Unlock() + if configChanged(c.cfg, cfg) { + c.clearWriters() + } + c.cfg = cfg + return nil +} + +func configChanged(old, new Config) bool { + if len(old.Brokers) != len(new.Brokers) { + return true + } + sort.Strings(old.Brokers) + sort.Strings(new.Brokers) + for i, b := range old.Brokers { + if new.Brokers[i] != b { + return true + } + } + return old.UseSSL != new.UseSSL || + old.SSLCA != new.SSLCA || + old.SSLCert != new.SSLCert || + old.SSLKey != new.SSLKey +} + +func (c *Cluster) clearWriters() { + for t, w := range c.writers { + w.Close() + delete(c.writers, t) + } +} + +type Service struct { + mu sync.RWMutex + clusters map[string]*Cluster + diag Diagnostic +} + +func NewService(cs Configs, d Diagnostic) *Service { + clusters := make(map[string]*Cluster, len(cs)) + for _, c := range cs { + if c.InsecureSkipVerify { + d.InsecureSkipVerify() + } + clusters[c.ID] = NewCluster(c) + } + return &Service{ + diag: d, + clusters: clusters, + } +} + +func (s *Service) Cluster(id string) (*Cluster, bool) { + s.mu.RLock() + defer s.mu.RUnlock() + c, ok := s.clusters[id] + return c, ok +} +func (s *Service) Update(newConfigs []interface{}) error { + s.mu.Lock() + defer s.mu.Unlock() + + clusterExists := make(map[string]bool, len(s.clusters)) + + for _, nc := range newConfigs { + if c, ok := nc.(Config); ok { + if err := c.Validate(); err != nil { + return err + } + if c.Enabled { + if c.InsecureSkipVerify { + s.diag.InsecureSkipVerify() + } + cluster, ok := s.clusters[c.ID] + if !ok { + s.clusters[c.ID] = NewCluster(c) + } else { + if err := cluster.Update(c); err != nil { + return errors.Wrapf(err, "failed to update cluster %q", c.ID) + } + } + clusterExists[c.ID] = true + } else { + cluster, ok := s.clusters[c.ID] + if ok { + cluster.Close() + delete(s.clusters, c.ID) + } + } + } else { + return fmt.Errorf("unexpected config object type, got %T exp %T", nc, c) + } + } + + // Find any deleted clusters + for name, cluster := range s.clusters { + if !clusterExists[name] { + cluster.Close() + delete(s.clusters, name) + } + } + + return nil +} + +func (s *Service) Open() error { + return nil +} + +func (s *Service) Close() error { + return nil +} + +type testOptions struct { + Cluster string `json:"cluster"` + Topic string `json:"topic"` + Key string `json:"key"` + Message string `json:"message"` +} + +func (s *Service) TestOptions() interface{} { + return &testOptions{ + Cluster: "example", + Topic: "test", + Key: "key", + Message: "test kafka message", + } +} + +func (s *Service) Test(options interface{}) error { + o, ok := options.(*testOptions) + if !ok { + return fmt.Errorf("unexpected options type %t", options) + } + c, ok := s.Cluster(o.Cluster) + if !ok { + return fmt.Errorf("unknown cluster %q", o.Cluster) + } + return c.WriteMessage(o.Topic, []byte(o.Key), []byte(o.Message)) +} + +type HandlerConfig struct { + Cluster string `mapstructure:"cluster"` + Topic string `mapstructure:"topic"` + Template string `mapstructure:"template"` +} + +type handler struct { + s *Service + + cluster *Cluster + topic string + template *template.Template + + diag Diagnostic +} + +func (s *Service) Handler(c HandlerConfig, ctx ...keyvalue.T) (alert.Handler, error) { + cluster, ok := s.Cluster(c.Cluster) + if !ok { + return nil, fmt.Errorf("unknown cluster %q", c.Cluster) + } + var t *template.Template + if c.Template != "" { + var err error + t, err = template.New("kafka alert template").Parse(c.Template) + if err != nil { + return nil, errors.Wrap(err, "failed to parse template") + } + } + return &handler{ + s: s, + cluster: cluster, + topic: c.Topic, + template: t, + diag: s.diag.WithContext(ctx...), + }, nil +} + +func (h *handler) Handle(event alert.Event) { + body, err := h.prepareBody(event.AlertData()) + if err != nil { + h.diag.Error("failed to prepare kafka message body", err) + } + if err := h.cluster.WriteMessage(h.topic, []byte(event.State.ID), body); err != nil { + h.diag.Error("failed to write message to kafka", err) + } +} +func (h *handler) prepareBody(ad alert.Data) ([]byte, error) { + body := bytes.Buffer{} + if h.template != nil { + err := h.template.Execute(&body, ad) + if err != nil { + return nil, errors.Wrap(err, "failed to execute alert template") + } + } else { + err := json.NewEncoder(&body).Encode(ad) + if err != nil { + return nil, errors.Wrap(err, "failed to marshal alert data json") + } + } + return body.Bytes(), nil +} diff --git a/task_master.go b/task_master.go index 636f0e5b9..f269aaaee 100644 --- a/task_master.go +++ b/task_master.go @@ -24,6 +24,7 @@ import ( "github.com/influxdata/kapacitor/services/httpd" "github.com/influxdata/kapacitor/services/httppost" k8s "github.com/influxdata/kapacitor/services/k8s/client" + "github.com/influxdata/kapacitor/services/kafka" "github.com/influxdata/kapacitor/services/mqtt" "github.com/influxdata/kapacitor/services/opsgenie" "github.com/influxdata/kapacitor/services/opsgenie2" @@ -166,6 +167,9 @@ type TaskMaster struct { StateChangesOnly() bool Handler(hipchat.HandlerConfig, ...keyvalue.T) alert.Handler } + KafkaService interface { + Handler(kafka.HandlerConfig, ...keyvalue.T) (alert.Handler, error) + } AlertaService interface { DefaultHandlerConfig() alerta.HandlerConfig Handler(alerta.HandlerConfig, ...keyvalue.T) (alert.Handler, error) diff --git a/vendor/github.com/segmentio/kafka-go/LICENSE b/vendor/github.com/segmentio/kafka-go/LICENSE new file mode 100644 index 000000000..09e136c51 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/LICENSE @@ -0,0 +1,21 @@ +MIT License + +Copyright (c) 2017 Segment + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. diff --git a/vendor/github.com/segmentio/kafka-go/balancer.go b/vendor/github.com/segmentio/kafka-go/balancer.go new file mode 100644 index 000000000..cac92417e --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/balancer.go @@ -0,0 +1,160 @@ +package kafka + +import ( + "hash" + "hash/fnv" + "sort" + "sync" +) + +// The Balancer interface provides an abstraction of the message distribution +// logic used by Writer instances to route messages to the partitions available +// on a kafka cluster. +// +// Instances of Balancer do not have to be safe to use concurrently by multiple +// goroutines, the Writer implementation ensures that calls to Balance are +// synchronized. +type Balancer interface { + // Balance receives a message and a set of available partitions and + // returns the partition number that the message should be routed to. + // + // An application should refrain from using a balancer to manage multiple + // sets of partitions (from different topics for examples), use one balancer + // instance for each partition set, so the balancer can detect when the + // partitions change and assume that the kafka topic has been rebalanced. + Balance(msg Message, partitions ...int) (partition int) +} + +// BalancerFunc is an implementation of the Balancer interface that makes it +// possible to use regular functions to distribute messages across partitions. +type BalancerFunc func(Message, ...int) int + +// Balance calls f, satisfies the Balancer interface. +func (f BalancerFunc) Balance(msg Message, partitions ...int) int { + return f(msg, partitions...) +} + +// RoundRobin is an Balancer implementation that equally distributes messages +// across all available partitions. +type RoundRobin struct { + offset uint64 +} + +// Balance satisfies the Balancer interface. +func (rr *RoundRobin) Balance(msg Message, partitions ...int) int { + length := uint64(len(partitions)) + offset := rr.offset + rr.offset++ + return partitions[offset%length] +} + +// LeastBytes is a Balancer implementation that routes messages to the partition +// that has received the least amount of data. +// +// Note that no coordination is done between multiple producers, having good +// balancing relies on the fact that each producer using a LeastBytes balancer +// should produce well balanced messages. +type LeastBytes struct { + counters []leastBytesCounter +} + +type leastBytesCounter struct { + partition int + bytes uint64 +} + +// Balance satisfies the Balancer interface. +func (lb *LeastBytes) Balance(msg Message, partitions ...int) int { + for _, p := range partitions { + if c := lb.counterOf(p); c == nil { + lb.counters = lb.makeCounters(partitions...) + break + } + } + + minBytes := lb.counters[0].bytes + minIndex := 0 + + for i, c := range lb.counters[1:] { + if c.bytes < minBytes { + minIndex = i + 1 + minBytes = c.bytes + } + } + + c := &lb.counters[minIndex] + c.bytes += uint64(len(msg.Key)) + uint64(len(msg.Value)) + return c.partition +} + +func (lb *LeastBytes) counterOf(partition int) *leastBytesCounter { + i := sort.Search(len(lb.counters), func(i int) bool { + return lb.counters[i].partition >= partition + }) + if i == len(lb.counters) || lb.counters[i].partition != partition { + return nil + } + return &lb.counters[i] +} + +func (lb *LeastBytes) makeCounters(partitions ...int) (counters []leastBytesCounter) { + counters = make([]leastBytesCounter, len(partitions)) + + for i, p := range partitions { + counters[i].partition = p + } + + sort.Slice(counters, func(i int, j int) bool { + return counters[i].partition < counters[j].partition + }) + return +} + +var ( + fnv1aPool = &sync.Pool{ + New: func() interface{} { + return fnv.New32a() + }, + } +) + +// Hash is a Balancer that uses the provided hash function to determine which +// partition to route messages to. This ensures that messages with the same key +// are routed to the same partition. +// +// The logic to calculate the partition is: +// +// hasher.Sum32() % len(partitions) => partition +// +// By default, Hash uses the FNV-1a algorithm. This is the same algorithm used +// by the Sarama Producer and ensures that messages produced by kafka-go will +// be delivered to the same topics that the Sarama producer would be delivered to +type Hash struct { + rr RoundRobin + Hasher hash.Hash32 +} + +func (h *Hash) Balance(msg Message, partitions ...int) (partition int) { + if msg.Key == nil { + return h.rr.Balance(msg, partitions...) + } + + hasher := h.Hasher + if hasher == nil { + hasher = fnv1aPool.Get().(hash.Hash32) + defer fnv1aPool.Put(hasher) + } + + hasher.Reset() + if _, err := hasher.Write(msg.Key); err != nil { + panic(err) + } + + // uses same algorithm that Sarama's hashPartitioner uses + partition = int(hasher.Sum32()) % len(partitions) + if partition < 0 { + partition = -partition + } + + return +} diff --git a/vendor/github.com/segmentio/kafka-go/batch.go b/vendor/github.com/segmentio/kafka-go/batch.go new file mode 100644 index 000000000..227024e85 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/batch.go @@ -0,0 +1,222 @@ +package kafka + +import ( + "bufio" + "io" + "sync" + "time" +) + +// A Batch is an iterator over a sequence of messages fetched from a kafka +// server. +// +// Batches are created by calling (*Conn).ReadBatch. They hold a internal lock +// on the connection, which is released when the batch is closed. Failing to +// call a batch's Close method will likely result in a dead-lock when trying to +// use the connection. +// +// Batches are safe to use concurrently from multiple goroutines. +type Batch struct { + mutex sync.Mutex + conn *Conn + lock *sync.Mutex + reader *bufio.Reader + deadline time.Time + throttle time.Duration + remain int + topic string + partition int + offset int64 + highWaterMark int64 + err error +} + +// Throttle gives the throttling duration applied by the kafka server on the +// connection. +func (batch *Batch) Throttle() time.Duration { + return batch.throttle +} + +// Watermark returns the current highest watermark in a partition. +func (batch *Batch) HighWaterMark() int64 { + return batch.highWaterMark +} + +// Offset returns the offset of the next message in the batch. +func (batch *Batch) Offset() int64 { + batch.mutex.Lock() + offset := batch.offset + batch.mutex.Unlock() + return offset +} + +// Close closes the batch, releasing the connection lock and returning an error +// if reading the batch failed for any reason. +func (batch *Batch) Close() error { + batch.mutex.Lock() + err := batch.close() + batch.mutex.Unlock() + return err +} + +func (batch *Batch) close() (err error) { + conn := batch.conn + lock := batch.lock + + batch.conn = nil + batch.lock = nil + batch.discard(batch.remain) + + if err = batch.err; err == io.EOF { + err = nil + } + + if conn != nil { + conn.rdeadline.unsetConnReadDeadline() + conn.mutex.Lock() + conn.offset = batch.offset + conn.mutex.Unlock() + + if err != nil { + if _, ok := err.(Error); !ok && err != io.ErrShortBuffer { + conn.Close() + } + } + } + + if lock != nil { + lock.Unlock() + } + + return +} + +// Read reads the value of the next message from the batch into b, returning the +// number of bytes read, or an error if the next message couldn't be read. +// +// If an error is returned the batch cannot be used anymore and calling Read +// again will keep returning that error. All errors except io.EOF (indicating +// that the program consumed all messages from the batch) are also returned by +// Close. +// +// The method fails with io.ErrShortBuffer if the buffer passed as argument is +// too small to hold the message value. +func (batch *Batch) Read(b []byte) (int, error) { + n := 0 + + batch.mutex.Lock() + offset := batch.offset + + _, _, err := batch.readMessage( + func(r *bufio.Reader, size int, nbytes int) (int, error) { + if nbytes < 0 { + return size, nil + } + return discardN(r, size, nbytes) + }, + func(r *bufio.Reader, size int, nbytes int) (int, error) { + if nbytes < 0 { + return size, nil + } + n = nbytes // return value + if nbytes > len(b) { + nbytes = len(b) + } + nbytes, err := io.ReadFull(r, b[:nbytes]) + if err != nil { + return size - nbytes, err + } + return discardN(r, size-nbytes, n-nbytes) + }, + ) + + if err == nil && n > len(b) { + n, err = len(b), io.ErrShortBuffer + batch.err = io.ErrShortBuffer + batch.offset = offset // rollback + } + + batch.mutex.Unlock() + return n, err +} + +// ReadMessage reads and return the next message from the batch. +// +// Because this method allocate memory buffers for the message key and value +// it is less memory-efficient than Read, but has the advantage of never +// failing with io.ErrShortBuffer. +func (batch *Batch) ReadMessage() (Message, error) { + msg := Message{} + batch.mutex.Lock() + + offset, timestamp, err := batch.readMessage( + func(r *bufio.Reader, size int, nbytes int) (remain int, err error) { + msg.Key, remain, err = readNewBytes(r, size, nbytes) + return + }, + func(r *bufio.Reader, size int, nbytes int) (remain int, err error) { + msg.Value, remain, err = readNewBytes(r, size, nbytes) + return + }, + ) + + batch.mutex.Unlock() + msg.Topic = batch.topic + msg.Partition = batch.partition + msg.Offset = offset + msg.Time = timestampToTime(timestamp) + return msg, err +} + +func (batch *Batch) readMessage( + key func(*bufio.Reader, int, int) (int, error), + val func(*bufio.Reader, int, int) (int, error), +) (offset int64, timestamp int64, err error) { + if err = batch.err; err != nil { + return + } + + offset, timestamp, batch.remain, err = readMessage( + batch.reader, + batch.remain, + batch.offset, + key, val, + ) + + switch err { + case nil: + batch.offset = offset + 1 + case errShortRead: + // As an "optimization" kafka truncates the returned response after + // producing MaxBytes, which could then cause the code to return + // errShortRead. + err = batch.discard(batch.remain) + default: + batch.err = err + } + + return +} + +func (batch *Batch) discard(n int) (err error) { + batch.remain, err = discardN(batch.reader, batch.remain, n) + switch { + case err != nil: + batch.err = err + case batch.err == nil && batch.remain == 0: + // Because we use the adjusted deadline we could end up returning + // before the actual deadline occurred. This is necessary otherwise + // timing out the connection for real could end up leaving it in an + // unpredictable state, which would require closing it. + // This design decision was main to maximize the changes of keeping + // the connection open, the trade off being to lose precision on the + // read deadline management. + if !batch.deadline.IsZero() && time.Now().After(batch.deadline) { + err = RequestTimedOut + } else { + err = io.EOF + } + batch.err = err + } + return +} diff --git a/vendor/github.com/segmentio/kafka-go/commit.go b/vendor/github.com/segmentio/kafka-go/commit.go new file mode 100644 index 000000000..e7740d58a --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/commit.go @@ -0,0 +1,39 @@ +package kafka + +// A commit represents the instruction of publishing an update of the last +// offset read by a program for a topic and partition. +type commit struct { + topic string + partition int + offset int64 +} + +// makeCommit builds a commit value from a message, the resulting commit takes +// its topic, partition, and offset from the message. +func makeCommit(msg Message) commit { + return commit{ + topic: msg.Topic, + partition: msg.Partition, + offset: msg.Offset + 1, + } +} + +// makeCommits generates a slice of commits from a list of messages, it extracts +// the topic, partition, and offset of each message and builds the corresponding +// commit slice. +func makeCommits(msgs ...Message) []commit { + commits := make([]commit, len(msgs)) + + for i, m := range msgs { + commits[i] = makeCommit(m) + } + + return commits +} + +// commitRequest is the data type exchanged between the CommitMessages method +// and internals of the reader's implementation. +type commitRequest struct { + commits []commit + errch chan<- error +} diff --git a/vendor/github.com/segmentio/kafka-go/conn.go b/vendor/github.com/segmentio/kafka-go/conn.go new file mode 100644 index 000000000..c6d829d1c --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/conn.go @@ -0,0 +1,1051 @@ +package kafka + +import ( + "bufio" + "errors" + "fmt" + "math" + "net" + "os" + "path/filepath" + "runtime" + "sync" + "sync/atomic" + "time" +) + +var ( + errInvalidWriteTopic = errors.New("writes must NOT set Topic on kafka.Message") + errInvalidWritePartition = errors.New("writes must NOT set Partition on kafka.Message") +) + +// Broker carries the metadata associated with a kafka broker. +type Broker struct { + Host string + Port int + ID int +} + +// Partition carries the metadata associated with a kafka partition. +type Partition struct { + Topic string + Leader Broker + Replicas []Broker + Isr []Broker + ID int +} + +// Conn represents a connection to a kafka broker. +// +// Instances of Conn are safe to use concurrently from multiple goroutines. +type Conn struct { + // base network connection + conn net.Conn + + // offset management (synchronized on the mutex field) + mutex sync.Mutex + offset int64 + + // read buffer (synchronized on rlock) + rlock sync.Mutex + rbuf bufio.Reader + + // write buffer (synchronized on wlock) + wlock sync.Mutex + wbuf bufio.Writer + + // deadline management + wdeadline connDeadline + rdeadline connDeadline + + // immutable values of the connection object + clientID string + topic string + partition int32 + fetchMaxBytes int32 + fetchMinSize int32 + + // correlation ID generator (synchronized on wlock) + correlationID int32 + + // number of replica acks required when publishing to a partition + requiredAcks int32 +} + +// ConnConfig is a configuration object used to create new instances of Conn. +type ConnConfig struct { + ClientID string + Topic string + Partition int +} + +var ( + // DefaultClientID is the default value used as ClientID of kafka + // connections. + DefaultClientID string +) + +func init() { + progname := filepath.Base(os.Args[0]) + hostname, _ := os.Hostname() + DefaultClientID = fmt.Sprintf("%s@%s (github.com/segmentio/kafka-go)", progname, hostname) +} + +// NewConn returns a new kafka connection for the given topic and partition. +func NewConn(conn net.Conn, topic string, partition int) *Conn { + return NewConnWith(conn, ConnConfig{ + Topic: topic, + Partition: partition, + }) +} + +// NewConnWith returns a new kafka connection configured with config. +func NewConnWith(conn net.Conn, config ConnConfig) *Conn { + if len(config.ClientID) == 0 { + config.ClientID = DefaultClientID + } + + if config.Partition < 0 || config.Partition > math.MaxInt32 { + panic(fmt.Sprintf("invalid partition number: %d", config.Partition)) + } + + c := &Conn{ + conn: conn, + rbuf: *bufio.NewReader(conn), + wbuf: *bufio.NewWriter(conn), + clientID: config.ClientID, + topic: config.Topic, + partition: int32(config.Partition), + offset: -2, + requiredAcks: -1, + } + + // The fetch request needs to ask for a MaxBytes value that is at least + // enough to load the control data of the response. To avoid having to + // recompute it on every read, it is cached here in the Conn value. + c.fetchMinSize = (fetchResponseV1{ + Topics: []fetchResponseTopicV1{{ + TopicName: config.Topic, + Partitions: []fetchResponsePartitionV1{{ + Partition: int32(config.Partition), + MessageSet: messageSet{{}}, + }}, + }}, + }).size() + c.fetchMaxBytes = math.MaxInt32 - c.fetchMinSize + return c +} + +// DeleteTopics deletes the specified topics. +func (c *Conn) DeleteTopics(topics ...string) error { + _, err := c.deleteTopics(deleteTopicsRequestV1{ + Topics: topics, + }) + return err +} + +// describeGroups retrieves the specified groups +// +// See http://kafka.apache.org/protocol.html#The_Messages_DescribeGroups +func (c *Conn) describeGroups(request describeGroupsRequestV1) (describeGroupsResponseV1, error) { + var response describeGroupsResponseV1 + + err := c.readOperation( + func(deadline time.Time, id int32) error { + return c.writeRequest(describeGroupsRequest, v1, id, request) + }, + func(deadline time.Time, size int) error { + return expectZeroSize(func() (remain int, err error) { + return (&response).readFrom(&c.rbuf, size) + }()) + }, + ) + if err != nil { + return describeGroupsResponseV1{}, err + } + for _, group := range response.Groups { + if group.ErrorCode != 0 { + return describeGroupsResponseV1{}, Error(group.ErrorCode) + } + } + + return response, nil +} + +// findCoordinator finds the coordinator for the specified group or transaction +// +// See http://kafka.apache.org/protocol.html#The_Messages_FindCoordinator +func (c *Conn) findCoordinator(request findCoordinatorRequestV1) (findCoordinatorResponseV1, error) { + var response findCoordinatorResponseV1 + + err := c.readOperation( + func(deadline time.Time, id int32) error { + return c.writeRequest(groupCoordinatorRequest, v1, id, request) + }, + func(deadline time.Time, size int) error { + return expectZeroSize(func() (remain int, err error) { + return (&response).readFrom(&c.rbuf, size) + }()) + }, + ) + if err != nil { + return findCoordinatorResponseV1{}, err + } + if response.ErrorCode != 0 { + return findCoordinatorResponseV1{}, Error(response.ErrorCode) + } + + return response, nil +} + +// heartbeat sends a heartbeat message required by consumer groups +// +// See http://kafka.apache.org/protocol.html#The_Messages_Heartbeat +func (c *Conn) heartbeat(request heartbeatRequestV1) (heartbeatResponseV1, error) { + var response heartbeatResponseV1 + + err := c.writeOperation( + func(deadline time.Time, id int32) error { + return c.writeRequest(heartbeatRequest, v1, id, request) + }, + func(deadline time.Time, size int) error { + return expectZeroSize(func() (remain int, err error) { + return (&response).readFrom(&c.rbuf, size) + }()) + }, + ) + if err != nil { + return heartbeatResponseV1{}, err + } + if response.ErrorCode != 0 { + return heartbeatResponseV1{}, Error(response.ErrorCode) + } + + return response, nil +} + +// joinGroup attempts to join a consumer group +// +// See http://kafka.apache.org/protocol.html#The_Messages_JoinGroup +func (c *Conn) joinGroup(request joinGroupRequestV2) (joinGroupResponseV2, error) { + var response joinGroupResponseV2 + + err := c.writeOperation( + func(deadline time.Time, id int32) error { + return c.writeRequest(joinGroupRequest, v2, id, request) + }, + func(deadline time.Time, size int) error { + return expectZeroSize(func() (remain int, err error) { + return (&response).readFrom(&c.rbuf, size) + }()) + }, + ) + if err != nil { + return joinGroupResponseV2{}, err + } + if response.ErrorCode != 0 { + return joinGroupResponseV2{}, Error(response.ErrorCode) + } + + return response, nil +} + +// leaveGroup leaves the consumer from the consumer group +// +// See http://kafka.apache.org/protocol.html#The_Messages_LeaveGroup +func (c *Conn) leaveGroup(request leaveGroupRequestV1) (leaveGroupResponseV1, error) { + var response leaveGroupResponseV1 + + err := c.writeOperation( + func(deadline time.Time, id int32) error { + return c.writeRequest(leaveGroupRequest, v1, id, request) + }, + func(deadline time.Time, size int) error { + return expectZeroSize(func() (remain int, err error) { + return (&response).readFrom(&c.rbuf, size) + }()) + }, + ) + if err != nil { + return leaveGroupResponseV1{}, err + } + if response.ErrorCode != 0 { + return leaveGroupResponseV1{}, Error(response.ErrorCode) + } + + return response, nil +} + +// listGroups lists all the consumer groups +// +// See http://kafka.apache.org/protocol.html#The_Messages_ListGroups +func (c *Conn) listGroups(request listGroupsRequestV1) (listGroupsResponseV1, error) { + var response listGroupsResponseV1 + + err := c.readOperation( + func(deadline time.Time, id int32) error { + return c.writeRequest(listGroupsRequest, v1, id, request) + }, + func(deadline time.Time, size int) error { + return expectZeroSize(func() (remain int, err error) { + return (&response).readFrom(&c.rbuf, size) + }()) + }, + ) + if err != nil { + return listGroupsResponseV1{}, err + } + if response.ErrorCode != 0 { + return listGroupsResponseV1{}, Error(response.ErrorCode) + } + + return response, nil +} + +// offsetCommit commits the specified topic partition offsets +// +// See http://kafka.apache.org/protocol.html#The_Messages_OffsetCommit +func (c *Conn) offsetCommit(request offsetCommitRequestV3) (offsetCommitResponseV3, error) { + var response offsetCommitResponseV3 + + err := c.writeOperation( + func(deadline time.Time, id int32) error { + return c.writeRequest(offsetCommitRequest, v3, id, request) + }, + func(deadline time.Time, size int) error { + return expectZeroSize(func() (remain int, err error) { + return (&response).readFrom(&c.rbuf, size) + }()) + }, + ) + if err != nil { + return offsetCommitResponseV3{}, err + } + for _, r := range response.Responses { + for _, pr := range r.PartitionResponses { + if pr.ErrorCode != 0 { + return offsetCommitResponseV3{}, Error(pr.ErrorCode) + } + } + } + + return response, nil +} + +// offsetFetch fetches the offsets for the specified topic partitions +// +// See http://kafka.apache.org/protocol.html#The_Messages_OffsetFetch +func (c *Conn) offsetFetch(request offsetFetchRequestV3) (offsetFetchResponseV3, error) { + var response offsetFetchResponseV3 + + err := c.readOperation( + func(deadline time.Time, id int32) error { + return c.writeRequest(offsetFetchRequest, v3, id, request) + }, + func(deadline time.Time, size int) error { + return expectZeroSize(func() (remain int, err error) { + return (&response).readFrom(&c.rbuf, size) + }()) + }, + ) + if err != nil { + return offsetFetchResponseV3{}, err + } + if response.ErrorCode != 0 { + return offsetFetchResponseV3{}, Error(response.ErrorCode) + } + for _, r := range response.Responses { + for _, pr := range r.PartitionResponses { + if pr.ErrorCode != 0 { + return offsetFetchResponseV3{}, Error(pr.ErrorCode) + } + } + } + + return response, nil +} + +// syncGroups completes the handshake to join a consumer group +// +// See http://kafka.apache.org/protocol.html#The_Messages_SyncGroup +func (c *Conn) syncGroups(request syncGroupRequestV1) (syncGroupResponseV1, error) { + var response syncGroupResponseV1 + + err := c.readOperation( + func(deadline time.Time, id int32) error { + return c.writeRequest(syncGroupRequest, v1, id, request) + }, + func(deadline time.Time, size int) error { + return expectZeroSize(func() (remain int, err error) { + return (&response).readFrom(&c.rbuf, size) + }()) + }, + ) + if err != nil { + return syncGroupResponseV1{}, err + } + if response.ErrorCode != 0 { + return syncGroupResponseV1{}, Error(response.ErrorCode) + } + + return response, nil +} + +// Close closes the kafka connection. +func (c *Conn) Close() error { + return c.conn.Close() +} + +// LocalAddr returns the local network address. +func (c *Conn) LocalAddr() net.Addr { + return c.conn.LocalAddr() +} + +// RemoteAddr returns the remote network address. +func (c *Conn) RemoteAddr() net.Addr { + return c.conn.RemoteAddr() +} + +// SetDeadline sets the read and write deadlines associated with the connection. +// It is equivalent to calling both SetReadDeadline and SetWriteDeadline. +// +// A deadline is an absolute time after which I/O operations fail with a timeout +// (see type Error) instead of blocking. The deadline applies to all future and +// pending I/O, not just the immediately following call to Read or Write. After +// a deadline has been exceeded, the connection may be closed if it was found to +// be in an unrecoverable state. +// +// A zero value for t means I/O operations will not time out. +func (c *Conn) SetDeadline(t time.Time) error { + c.rdeadline.setDeadline(t) + c.wdeadline.setDeadline(t) + return nil +} + +// SetReadDeadline sets the deadline for future Read calls and any +// currently-blocked Read call. +// A zero value for t means Read will not time out. +func (c *Conn) SetReadDeadline(t time.Time) error { + c.rdeadline.setDeadline(t) + return nil +} + +// SetWriteDeadline sets the deadline for future Write calls and any +// currently-blocked Write call. +// Even if write times out, it may return n > 0, indicating that some of the +// data was successfully written. +// A zero value for t means Write will not time out. +func (c *Conn) SetWriteDeadline(t time.Time) error { + c.wdeadline.setDeadline(t) + return nil +} + +// Offset returns the current offset of the connection as pair of integers, +// where the first one is an offset value and the second one indicates how +// to interpret it. +// +// See Seek for more details about the offset and whence values. +func (c *Conn) Offset() (offset int64, whence int) { + c.mutex.Lock() + offset = c.offset + c.mutex.Unlock() + switch offset { + case -1: + offset = 0 + whence = 2 + case -2: + offset = 0 + whence = 0 + default: + whence = 1 + } + return +} + +// Seek changes the offset of the connection to offset, interpreted according to +// whence: 0 means relative to the first offset, 1 means relative to the current +// offset, and 2 means relative to the last offset. +// The method returns the new absoluate offset of the connection. +func (c *Conn) Seek(offset int64, whence int) (int64, error) { + switch whence { + case 0, 1, 2: + default: + return 0, fmt.Errorf("the whence value has to be 0, 1, or 2 (whence = %d)", whence) + } + + if whence == 1 { + c.mutex.Lock() + unchanged := offset == c.offset + c.mutex.Unlock() + if unchanged { + return offset, nil + } + } + + first, last, err := c.ReadOffsets() + if err != nil { + return 0, err + } + + switch whence { + case 0: + offset = first + offset + case 2: + offset = last - offset + } + + if offset < first || offset > last { + return 0, OffsetOutOfRange + } + + c.mutex.Lock() + c.offset = offset + c.mutex.Unlock() + return offset, nil +} + +// Read reads the message at the current offset from the connection, advancing +// the offset on success so the next call to a read method will produce the next +// message. +// The method returns the number of bytes read, or an error if something went +// wrong. +// +// While it is safe to call Read concurrently from multiple goroutines it may +// be hard for the program to predict the results as the connection offset will +// be read and written by multiple goroutines, they could read duplicates, or +// messages may be seen by only some of the goroutines. +// +// The method fails with io.ErrShortBuffer if the buffer passed as argument is +// too small to hold the message value. +// +// This method is provided to satisfy the net.Conn interface but is much less +// efficient than using the more general purpose ReadBatch method. +func (c *Conn) Read(b []byte) (int, error) { + batch := c.ReadBatch(1, len(b)) + n, err := batch.Read(b) + return n, coalesceErrors(silentEOF(err), batch.Close()) +} + +// ReadMessage reads the message at the current offset from the connection, +// advancing the offset on success so the next call to a read method will +// produce the next message. +// +// Because this method allocate memory buffers for the message key and value +// it is less memory-efficient than Read, but has the advantage of never +// failing with io.ErrShortBuffer. +// +// While it is safe to call Read concurrently from multiple goroutines it may +// be hard for the program to predict the results as the connection offset will +// be read and written by multiple goroutines, they could read duplicates, or +// messages may be seen by only some of the goroutines. +// +// This method is provided for convenience purposes but is much less efficient +// than using the more general purpose ReadBatch method. +func (c *Conn) ReadMessage(maxBytes int) (Message, error) { + batch := c.ReadBatch(1, maxBytes) + msg, err := batch.ReadMessage() + return msg, coalesceErrors(silentEOF(err), batch.Close()) +} + +// ReadBatch reads a batch of messages from the kafka server. The method always +// returns a non-nil Batch value. If an error occurred, either sending the fetch +// request or reading the response, the error will be made available by the +// returned value of the batch's Close method. +// +// While it is safe to call ReadBatch concurrently from multiple goroutines it +// may be hard for the program to predict the results as the connection offset +// will be read and written by multiple goroutines, they could read duplicates, +// or messages may be seen by only some of the goroutines. +// +// A program doesn't specify the number of messages in wants from a batch, but +// gives the minimum and maximum number of bytes that it wants to receive from +// the kafka server. +func (c *Conn) ReadBatch(minBytes int, maxBytes int) *Batch { + var adjustedDeadline time.Time + var maxFetch = int(c.fetchMaxBytes) + + if minBytes < 0 || minBytes > maxFetch { + return &Batch{err: fmt.Errorf("kafka.(*Conn).ReadBatch: minBytes of %d out of [1,%d] bounds", minBytes, maxFetch)} + } + if maxBytes < 0 || maxBytes > maxFetch { + return &Batch{err: fmt.Errorf("kafka.(*Conn).ReadBatch: maxBytes of %d out of [1,%d] bounds", maxBytes, maxFetch)} + } + if minBytes > maxBytes { + return &Batch{err: fmt.Errorf("kafka.(*Conn).ReadBatch: minBytes (%d) > maxBytes (%d)", minBytes, maxBytes)} + } + + offset, err := c.Seek(c.Offset()) + if err != nil { + return &Batch{err: dontExpectEOF(err)} + } + + id, err := c.doRequest(&c.rdeadline, func(deadline time.Time, id int32) error { + now := time.Now() + deadline = adjustDeadlineForRTT(deadline, now, defaultRTT) + adjustedDeadline = deadline + return writeFetchRequestV1( + &c.wbuf, + id, + c.clientID, + c.topic, + c.partition, + offset, + minBytes, + maxBytes+int(c.fetchMinSize), + deadlineToTimeout(deadline, now), + ) + }) + if err != nil { + return &Batch{err: dontExpectEOF(err)} + } + + _, size, lock, err := c.waitResponse(&c.rdeadline, id) + if err != nil { + return &Batch{err: dontExpectEOF(err)} + } + + throttle, highWaterMark, remain, err := readFetchResponseHeader(&c.rbuf, size) + return &Batch{ + conn: c, + reader: &c.rbuf, + deadline: adjustedDeadline, + throttle: duration(throttle), + lock: lock, + remain: remain, + topic: c.topic, // topic is copied to Batch to prevent race with Batch.close + partition: int(c.partition), // partition is copied to Batch to prevent race with Batch.close + offset: offset, + highWaterMark: highWaterMark, + err: dontExpectEOF(err), + } +} + +// ReadOffset returns the offset of the first message with a timestamp equal or +// greater to t. +func (c *Conn) ReadOffset(t time.Time) (int64, error) { + return c.readOffset(timestamp(t)) +} + +// ReadFirstOffset returns the first offset available on the connection. +func (c *Conn) ReadFirstOffset() (int64, error) { + return c.readOffset(-2) +} + +// ReadLastOffset returns the last offset available on the connection. +func (c *Conn) ReadLastOffset() (int64, error) { + return c.readOffset(-1) +} + +// ReadOffsets returns the absolute first and last offsets of the topic used by +// the connection. +func (c *Conn) ReadOffsets() (first int64, last int64, err error) { + // We have to submit two different requests to fetch the first and last + // offsets because kafka refuses requests that ask for multiple offsets + // on the same topic and partition. + if first, err = c.ReadFirstOffset(); err != nil { + return + } + if last, err = c.ReadLastOffset(); err != nil { + first = 0 // don't leak the value on error + return + } + return +} + +func (c *Conn) readOffset(t int64) (offset int64, err error) { + err = c.readOperation( + func(deadline time.Time, id int32) error { + return writeListOffsetRequestV1(&c.wbuf, id, c.clientID, c.topic, c.partition, t) + }, + func(deadline time.Time, size int) error { + return expectZeroSize(readArrayWith(&c.rbuf, size, func(r *bufio.Reader, size int) (int, error) { + // We skip the topic name because we've made a request for + // a single topic. + size, err := discardString(r, size) + if err != nil { + return size, err + } + + // Reading the array of partitions, there will be only one + // partition which gives the offset we're looking for. + return readArrayWith(r, size, func(r *bufio.Reader, size int) (int, error) { + var p partitionOffsetV1 + size, err := p.readFrom(r, size) + if err != nil { + return size, err + } + if p.ErrorCode != 0 { + return size, Error(p.ErrorCode) + } + offset = p.Offset + return size, nil + }) + })) + }, + ) + return +} + +// ReadPartitions returns the list of available partitions for the given list of +// topics. +// +// If the method is called with no topic, it uses the topic configured on the +// connection. If there are none, the method fetches all partitions of the kafka +// cluster. +func (c *Conn) ReadPartitions(topics ...string) (partitions []Partition, err error) { + defaultTopics := [...]string{c.topic} + + if len(topics) == 0 && len(c.topic) != 0 { + topics = defaultTopics[:] + } + + err = c.readOperation( + func(deadline time.Time, id int32) error { + return c.writeRequest(metadataRequest, v0, id, topicMetadataRequestV0(topics)) + }, + func(deadline time.Time, size int) error { + var res metadataResponseV0 + + if err := c.readResponse(size, &res); err != nil { + return err + } + + brokers := make(map[int32]Broker, len(res.Brokers)) + for _, b := range res.Brokers { + brokers[b.NodeID] = Broker{ + Host: b.Host, + Port: int(b.Port), + ID: int(b.NodeID), + } + } + + makeBrokers := func(ids ...int32) []Broker { + b := make([]Broker, len(ids)) + for i, id := range ids { + b[i] = brokers[id] + } + return b + } + + for _, t := range res.Topics { + if t.TopicErrorCode != 0 && t.TopicName == c.topic { + // We only report errors if they happened for the topic of + // the connection, otherwise the topic will simply have no + // partitions in the result set. + return Error(t.TopicErrorCode) + } + for _, p := range t.Partitions { + partitions = append(partitions, Partition{ + Topic: t.TopicName, + Leader: brokers[p.Leader], + Replicas: makeBrokers(p.Replicas...), + Isr: makeBrokers(p.Isr...), + ID: int(p.PartitionID), + }) + } + } + return nil + }, + ) + return +} + +// Write writes a message to the kafka broker that this connection was +// established to. The method returns the number of bytes written, or an error +// if something went wrong. +// +// The operation either succeeds or fail, it never partially writes the message. +// +// This method is exposed to satisfy the net.Conn interface but is less efficient +// than the more general purpose WriteMessages method. +func (c *Conn) Write(b []byte) (int, error) { + return c.WriteMessages(Message{Value: b}) +} + +// WriteMessages writes a batch of messages to the connection's topic and +// partition, returning the number of bytes written. The write is an atomic +// operation, it either fully succeeds or fails. +func (c *Conn) WriteMessages(msgs ...Message) (int, error) { + if len(msgs) == 0 { + return 0, nil + } + + writeTime := time.Now() + for _, msg := range msgs { + // users may believe they can set the Topic and/or Partition + // on the kafka message. + if msg.Topic != "" && msg.Topic != c.topic { + return 0, errInvalidWriteTopic + } + if msg.Partition != 0 { + return 0, errInvalidWritePartition + } + + if msg.Time.IsZero() { + msg.Time = writeTime + } + } + + n := 0 + for _, msg := range msgs { + n += len(msg.Key) + len(msg.Value) + } + + err := c.writeOperation( + func(deadline time.Time, id int32) error { + now := time.Now() + deadline = adjustDeadlineForRTT(deadline, now, defaultRTT) + return writeProduceRequestV2( + &c.wbuf, + id, + c.clientID, + c.topic, + c.partition, + deadlineToTimeout(deadline, now), + int16(atomic.LoadInt32(&c.requiredAcks)), + msgs..., + ) + }, + func(deadline time.Time, size int) error { + return expectZeroSize(readArrayWith(&c.rbuf, size, func(r *bufio.Reader, size int) (int, error) { + // Skip the topic, we've produced the message to only one topic, + // no need to waste resources loading it in memory. + size, err := discardString(r, size) + if err != nil { + return size, err + } + + // Read the list of partitions, there should be only one since + // we've produced a message to a single partition. + size, err = readArrayWith(r, size, func(r *bufio.Reader, size int) (int, error) { + var p produceResponsePartitionV2 + size, err := p.readFrom(r, size) + if err == nil && p.ErrorCode != 0 { + err = Error(p.ErrorCode) + } + return size, err + }) + if err != nil { + return size, err + } + + // The response is trailed by the throttle time, also skipping + // since it's not interesting here. + return discardInt32(r, size) + })) + }, + ) + + if err != nil { + n = 0 + } + + return n, err +} + +// SetRequiredAcks sets the number of acknowledges from replicas that the +// connection requests when producing messages. +func (c *Conn) SetRequiredAcks(n int) error { + switch n { + case -1, 1: + atomic.StoreInt32(&c.requiredAcks, int32(n)) + return nil + default: + return InvalidRequiredAcks + } +} + +func (c *Conn) writeRequestHeader(apiKey apiKey, apiVersion apiVersion, correlationID int32, size int32) { + hdr := c.requestHeader(apiKey, apiVersion, correlationID) + hdr.Size = (hdr.size() + size) - 4 + hdr.writeTo(&c.wbuf) +} + +func (c *Conn) writeRequest(apiKey apiKey, apiVersion apiVersion, correlationID int32, req request) error { + hdr := c.requestHeader(apiKey, apiVersion, correlationID) + hdr.Size = (hdr.size() + req.size()) - 4 + hdr.writeTo(&c.wbuf) + req.writeTo(&c.wbuf) + return c.wbuf.Flush() +} + +func (c *Conn) readResponse(size int, res interface{}) error { + size, err := read(&c.rbuf, size, res) + switch err.(type) { + case Error: + var e error + if size, e = discardN(&c.rbuf, size, size); e != nil { + err = e + } + } + return expectZeroSize(size, err) +} + +func (c *Conn) peekResponseSizeAndID() (int32, int32, error) { + b, err := c.rbuf.Peek(8) + if err != nil { + return 0, 0, err + } + size, id := makeInt32(b[:4]), makeInt32(b[4:]) + return size, id, nil +} + +func (c *Conn) skipResponseSizeAndID() { + c.rbuf.Discard(8) +} + +func (c *Conn) readDeadline() time.Time { + return c.rdeadline.deadline() +} + +func (c *Conn) writeDeadline() time.Time { + return c.wdeadline.deadline() +} + +func (c *Conn) readOperation(write func(time.Time, int32) error, read func(time.Time, int) error) error { + return c.do(&c.rdeadline, write, read) +} + +func (c *Conn) writeOperation(write func(time.Time, int32) error, read func(time.Time, int) error) error { + return c.do(&c.wdeadline, write, read) +} + +func (c *Conn) do(d *connDeadline, write func(time.Time, int32) error, read func(time.Time, int) error) error { + id, err := c.doRequest(d, write) + if err != nil { + return err + } + + deadline, size, lock, err := c.waitResponse(d, id) + if err != nil { + return err + } + + if err = read(deadline, size); err != nil { + switch err.(type) { + case Error: + default: + c.conn.Close() + } + } + + d.unsetConnReadDeadline() + lock.Unlock() + return err +} + +func (c *Conn) doRequest(d *connDeadline, write func(time.Time, int32) error) (id int32, err error) { + c.wlock.Lock() + c.correlationID++ + id = c.correlationID + err = write(d.setConnWriteDeadline(c.conn), id) + d.unsetConnWriteDeadline() + + if err != nil { + // When an error occurs there's no way to know if the connection is in a + // recoverable state so we're better off just giving up at this point to + // avoid any risk of corrupting the following operations. + c.conn.Close() + } + + c.wlock.Unlock() + return +} + +func (c *Conn) waitResponse(d *connDeadline, id int32) (deadline time.Time, size int, lock *sync.Mutex, err error) { + for { + var rsz int32 + var rid int32 + + c.rlock.Lock() + deadline = d.setConnReadDeadline(c.conn) + + if rsz, rid, err = c.peekResponseSizeAndID(); err != nil { + d.unsetConnReadDeadline() + c.conn.Close() + c.rlock.Unlock() + return + } + + if id == rid { + c.skipResponseSizeAndID() + size, lock = int(rsz-4), &c.rlock + return + } + + // Optimistically release the read lock if a response has already + // been received but the current operation is not the target for it. + c.rlock.Unlock() + runtime.Gosched() + } +} + +func (c *Conn) requestHeader(apiKey apiKey, apiVersion apiVersion, correlationID int32) requestHeader { + return requestHeader{ + ApiKey: int16(apiKey), + ApiVersion: int16(apiVersion), + CorrelationID: correlationID, + ClientID: c.clientID, + } +} + +// connDeadline is a helper type to implement read/write deadline management on +// the kafka connection. +type connDeadline struct { + mutex sync.Mutex + value time.Time + rconn net.Conn + wconn net.Conn +} + +func (d *connDeadline) deadline() time.Time { + d.mutex.Lock() + t := d.value + d.mutex.Unlock() + return t +} + +func (d *connDeadline) setDeadline(t time.Time) { + d.mutex.Lock() + d.value = t + + if d.rconn != nil { + d.rconn.SetReadDeadline(t) + } + + if d.wconn != nil { + d.wconn.SetWriteDeadline(t) + } + + d.mutex.Unlock() +} + +func (d *connDeadline) setConnReadDeadline(conn net.Conn) time.Time { + d.mutex.Lock() + deadline := d.value + d.rconn = conn + d.rconn.SetReadDeadline(deadline) + d.mutex.Unlock() + return deadline +} + +func (d *connDeadline) setConnWriteDeadline(conn net.Conn) time.Time { + d.mutex.Lock() + deadline := d.value + d.wconn = conn + d.wconn.SetWriteDeadline(deadline) + d.mutex.Unlock() + return deadline +} + +func (d *connDeadline) unsetConnReadDeadline() { + d.mutex.Lock() + d.rconn = nil + d.mutex.Unlock() +} + +func (d *connDeadline) unsetConnWriteDeadline() { + d.mutex.Lock() + d.wconn = nil + d.mutex.Unlock() +} diff --git a/vendor/github.com/segmentio/kafka-go/crc32.go b/vendor/github.com/segmentio/kafka-go/crc32.go new file mode 100644 index 000000000..f1a617f02 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/crc32.go @@ -0,0 +1,80 @@ +package kafka + +import ( + "bytes" + "encoding/binary" + "hash/crc32" + "sync" +) + +func crc32OfMessage(magicByte int8, attributes int8, timestamp int64, key []byte, value []byte) uint32 { + b := acquireCrc32Buffer() + b.writeInt8(magicByte) + b.writeInt8(attributes) + if magicByte != 0 { + b.writeInt64(timestamp) + } + b.writeBytes(key) + b.writeBytes(value) + sum := b.sum + releaseCrc32Buffer(b) + return sum +} + +type crc32Buffer struct { + sum uint32 + buf bytes.Buffer +} + +func (c *crc32Buffer) writeInt8(i int8) { + c.buf.Truncate(0) + c.buf.WriteByte(byte(i)) + c.update() +} + +func (c *crc32Buffer) writeInt32(i int32) { + a := [4]byte{} + binary.BigEndian.PutUint32(a[:], uint32(i)) + c.buf.Truncate(0) + c.buf.Write(a[:]) + c.update() +} + +func (c *crc32Buffer) writeInt64(i int64) { + a := [8]byte{} + binary.BigEndian.PutUint64(a[:], uint64(i)) + c.buf.Truncate(0) + c.buf.Write(a[:]) + c.update() +} + +func (c *crc32Buffer) writeBytes(b []byte) { + if b == nil { + c.writeInt32(-1) + } else { + c.writeInt32(int32(len(b))) + } + c.sum = crc32Update(c.sum, b) +} + +func (c *crc32Buffer) update() { + c.sum = crc32Update(c.sum, c.buf.Bytes()) +} + +func crc32Update(sum uint32, b []byte) uint32 { + return crc32.Update(sum, crc32.IEEETable, b) +} + +var crc32BufferPool = sync.Pool{ + New: func() interface{} { return &crc32Buffer{} }, +} + +func acquireCrc32Buffer() *crc32Buffer { + c := crc32BufferPool.Get().(*crc32Buffer) + c.sum = 0 + return c +} + +func releaseCrc32Buffer(b *crc32Buffer) { + crc32BufferPool.Put(b) +} diff --git a/vendor/github.com/segmentio/kafka-go/createtopics.go b/vendor/github.com/segmentio/kafka-go/createtopics.go new file mode 100644 index 000000000..a3cccb03d --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/createtopics.go @@ -0,0 +1,288 @@ +package kafka + +import ( + "bufio" + "time" +) + +type ConfigEntry struct { + ConfigName string + ConfigValue string +} + +func (c ConfigEntry) toCreateTopicsRequestV2ConfigEntry() createTopicsRequestV2ConfigEntry { + return createTopicsRequestV2ConfigEntry{ + ConfigName: c.ConfigName, + ConfigValue: c.ConfigValue, + } +} + +type createTopicsRequestV2ConfigEntry struct { + ConfigName string + ConfigValue string +} + +func (t createTopicsRequestV2ConfigEntry) size() int32 { + return sizeofString(t.ConfigName) + + sizeofString(t.ConfigValue) +} + +func (t createTopicsRequestV2ConfigEntry) writeTo(w *bufio.Writer) { + writeString(w, t.ConfigName) + writeString(w, t.ConfigValue) +} + +type ReplicaAssignment struct { + Partition int + Replicas int +} + +func (a ReplicaAssignment) toCreateTopicsRequestV2ReplicaAssignment() createTopicsRequestV2ReplicaAssignment { + return createTopicsRequestV2ReplicaAssignment{ + Partition: int32(a.Partition), + Replicas: int32(a.Replicas), + } +} + +type createTopicsRequestV2ReplicaAssignment struct { + Partition int32 + Replicas int32 +} + +func (t createTopicsRequestV2ReplicaAssignment) size() int32 { + return sizeofInt32(t.Partition) + + sizeofInt32(t.Replicas) +} + +func (t createTopicsRequestV2ReplicaAssignment) writeTo(w *bufio.Writer) { + writeInt32(w, t.Partition) + writeInt32(w, t.Replicas) +} + +type TopicConfig struct { + // Topic name + Topic string + + // NumPartitions created. -1 indicates unset. + NumPartitions int + + // ReplicationFactor for the topic. -1 indicates unset. + ReplicationFactor int + + // ReplicaAssignments among kafka brokers for this topic partitions. If this + // is set num_partitions and replication_factor must be unset. + ReplicaAssignments []ReplicaAssignment + + // ConfigEntries holds topic level configuration for topic to be set. + ConfigEntries []ConfigEntry +} + +func (t TopicConfig) toCreateTopicsRequestV2Topic() createTopicsRequestV2Topic { + var requestV2ReplicaAssignments []createTopicsRequestV2ReplicaAssignment + for _, a := range t.ReplicaAssignments { + requestV2ReplicaAssignments = append( + requestV2ReplicaAssignments, + a.toCreateTopicsRequestV2ReplicaAssignment()) + } + var requestV2ConfigEntries []createTopicsRequestV2ConfigEntry + for _, c := range t.ConfigEntries { + requestV2ConfigEntries = append( + requestV2ConfigEntries, + c.toCreateTopicsRequestV2ConfigEntry()) + } + + return createTopicsRequestV2Topic{ + Topic: t.Topic, + NumPartitions: int32(t.NumPartitions), + ReplicationFactor: int16(t.ReplicationFactor), + ReplicaAssignments: requestV2ReplicaAssignments, + ConfigEntries: requestV2ConfigEntries, + } +} + +type createTopicsRequestV2Topic struct { + // Topic name + Topic string + + // NumPartitions created. -1 indicates unset. + NumPartitions int32 + + // ReplicationFactor for the topic. -1 indicates unset. + ReplicationFactor int16 + + // ReplicaAssignments among kafka brokers for this topic partitions. If this + // is set num_partitions and replication_factor must be unset. + ReplicaAssignments []createTopicsRequestV2ReplicaAssignment + + // ConfigEntries holds topic level configuration for topic to be set. + ConfigEntries []createTopicsRequestV2ConfigEntry +} + +func (t createTopicsRequestV2Topic) size() int32 { + return sizeofString(t.Topic) + + sizeofInt32(t.NumPartitions) + + sizeofInt16(t.ReplicationFactor) + + sizeofArray(len(t.ReplicaAssignments), func(i int) int32 { return t.ReplicaAssignments[i].size() }) + + sizeofArray(len(t.ConfigEntries), func(i int) int32 { return t.ConfigEntries[i].size() }) +} + +func (t createTopicsRequestV2Topic) writeTo(w *bufio.Writer) { + writeString(w, t.Topic) + writeInt32(w, t.NumPartitions) + writeInt16(w, t.ReplicationFactor) + writeArray(w, len(t.ReplicaAssignments), func(i int) { t.ReplicaAssignments[i].writeTo(w) }) + writeArray(w, len(t.ConfigEntries), func(i int) { t.ConfigEntries[i].writeTo(w) }) +} + +// See http://kafka.apache.org/protocol.html#The_Messages_CreateTopics +type createTopicsRequestV2 struct { + // Topics contains n array of single topic creation requests. Can not + // have multiple entries for the same topic. + Topics []createTopicsRequestV2Topic + + // Timeout ms to wait for a topic to be completely created on the + // controller node. Values <= 0 will trigger topic creation and return immediately + Timeout int32 + + // ValidateOnly if true, the request will be validated, but the topic won + // 't be created. + ValidateOnly bool +} + +func (t createTopicsRequestV2) size() int32 { + return sizeofArray(len(t.Topics), func(i int) int32 { return t.Topics[i].size() }) + + sizeofInt32(t.Timeout) + + sizeofBool(t.ValidateOnly) +} + +func (t createTopicsRequestV2) writeTo(w *bufio.Writer) { + writeArray(w, len(t.Topics), func(i int) { t.Topics[i].writeTo(w) }) + writeInt32(w, t.Timeout) + writeBool(w, t.ValidateOnly) +} + +type createTopicsResponseV2TopicError struct { + // Topic name + Topic string + + // ErrorCode holds response error code + ErrorCode int16 + + // ErrorMessage holds the response error message + ErrorMessage string +} + +func (t createTopicsResponseV2TopicError) size() int32 { + return sizeofString(t.Topic) + + sizeofInt16(t.ErrorCode) + + sizeofString(t.ErrorMessage) +} + +func (t createTopicsResponseV2TopicError) writeTo(w *bufio.Writer) { + writeString(w, t.Topic) + writeInt16(w, t.ErrorCode) + writeString(w, t.ErrorMessage) +} + +func (t *createTopicsResponseV2TopicError) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readString(r, size, &t.Topic); err != nil { + return + } + if remain, err = readInt16(r, remain, &t.ErrorCode); err != nil { + return + } + if remain, err = readString(r, remain, &t.ErrorMessage); err != nil { + return + } + return +} + +// See http://kafka.apache.org/protocol.html#The_Messages_CreateTopics +type createTopicsResponseV2 struct { + ThrottleTimeMS int32 + TopicErrors []createTopicsResponseV2TopicError +} + +func (t createTopicsResponseV2) size() int32 { + return sizeofInt32(t.ThrottleTimeMS) + + sizeofArray(len(t.TopicErrors), func(i int) int32 { return t.TopicErrors[i].size() }) +} + +func (t createTopicsResponseV2) writeTo(w *bufio.Writer) { + writeInt32(w, t.ThrottleTimeMS) + writeArray(w, len(t.TopicErrors), func(i int) { t.TopicErrors[i].writeTo(w) }) +} + +func (t *createTopicsResponseV2) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readInt32(r, size, &t.ThrottleTimeMS); err != nil { + return + } + + fn := func(r *bufio.Reader, size int) (fnRemain int, fnErr error) { + var topic createTopicsResponseV2TopicError + if fnRemain, fnErr = (&topic).readFrom(r, size); err != nil { + return + } + t.TopicErrors = append(t.TopicErrors, topic) + return + } + if remain, err = readArrayWith(r, remain, fn); err != nil { + return + } + + return +} + +func (c *Conn) createTopics(request createTopicsRequestV2) (createTopicsResponseV2, error) { + var response createTopicsResponseV2 + + err := c.writeOperation( + func(deadline time.Time, id int32) error { + if request.Timeout == 0 { + now := time.Now() + deadline = adjustDeadlineForRTT(deadline, now, defaultRTT) + request.Timeout = milliseconds(deadlineToTimeout(deadline, now)) + } + return c.writeRequest(createTopicsRequest, v2, id, request) + }, + func(deadline time.Time, size int) error { + return expectZeroSize(func() (remain int, err error) { + return (&response).readFrom(&c.rbuf, size) + }()) + }, + ) + if err != nil { + return response, err + } + for _, tr := range response.TopicErrors { + if tr.ErrorCode != 0 { + return response, Error(tr.ErrorCode) + } + } + + return response, nil +} + +// CreateTopics creates one topic per provided configuration with idempotent +// operational semantics. In other words, if CreateTopics is invoked with a +// configuration for an existing topic, it will have no effect. +func (c *Conn) CreateTopics(topics ...TopicConfig) error { + var requestV2Topics []createTopicsRequestV2Topic + for _, t := range topics { + requestV2Topics = append( + requestV2Topics, + t.toCreateTopicsRequestV2Topic()) + } + + _, err := c.createTopics(createTopicsRequestV2{ + Topics: requestV2Topics, + }) + + switch err { + case TopicAlreadyExists: + // ok + return nil + default: + return err + } +} diff --git a/vendor/github.com/segmentio/kafka-go/deletetopics.go b/vendor/github.com/segmentio/kafka-go/deletetopics.go new file mode 100644 index 000000000..983bff7a0 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/deletetopics.go @@ -0,0 +1,124 @@ +package kafka + +import ( + "bufio" + "time" +) + +// See http://kafka.apache.org/protocol.html#The_Messages_DeleteTopics +type deleteTopicsRequestV1 struct { + // Topics holds the topic names + Topics []string + + // Timeout holds the time in ms to wait for a topic to be completely deleted + // on the controller node. Values <= 0 will trigger topic deletion and return + // immediately. + Timeout int32 +} + +func (t deleteTopicsRequestV1) size() int32 { + return sizeofStringArray(t.Topics) + + sizeofInt32(t.Timeout) +} + +func (t deleteTopicsRequestV1) writeTo(w *bufio.Writer) { + writeStringArray(w, t.Topics) + writeInt32(w, t.Timeout) +} + +type deleteTopicsResponseV1 struct { + // ThrottleTimeMS holds the duration in milliseconds for which the request + // was throttled due to quota violation (Zero if the request did not violate + // any quota) + ThrottleTimeMS int32 + + // TopicErrorCodes holds per topic error codes + TopicErrorCodes []deleteTopicsResponseV1TopicErrorCode +} + +func (t deleteTopicsResponseV1) size() int32 { + return sizeofInt32(t.ThrottleTimeMS) + + sizeofArray(len(t.TopicErrorCodes), func(i int) int32 { return t.TopicErrorCodes[i].size() }) +} + +func (t *deleteTopicsResponseV1) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readInt32(r, size, &t.ThrottleTimeMS); err != nil { + return + } + fn := func(withReader *bufio.Reader, withSize int) (fnRemain int, fnErr error) { + var item deleteTopicsResponseV1TopicErrorCode + if fnRemain, fnErr = (&item).readFrom(withReader, withSize); err != nil { + return + } + t.TopicErrorCodes = append(t.TopicErrorCodes, item) + return + } + if remain, err = readArrayWith(r, remain, fn); err != nil { + return + } + return +} + +func (t deleteTopicsResponseV1) writeTo(w *bufio.Writer) { + writeInt32(w, t.ThrottleTimeMS) + writeArray(w, len(t.TopicErrorCodes), func(i int) { t.TopicErrorCodes[i].writeTo(w) }) +} + +type deleteTopicsResponseV1TopicErrorCode struct { + // Topic holds the topic name + Topic string + + // ErrorCode holds the error code + ErrorCode int16 +} + +func (t deleteTopicsResponseV1TopicErrorCode) size() int32 { + return sizeofString(t.Topic) + + sizeofInt16(t.ErrorCode) +} + +func (t *deleteTopicsResponseV1TopicErrorCode) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readString(r, size, &t.Topic); err != nil { + return + } + if remain, err = readInt16(r, remain, &t.ErrorCode); err != nil { + return + } + return +} + +func (t deleteTopicsResponseV1TopicErrorCode) writeTo(w *bufio.Writer) { + writeString(w, t.Topic) + writeInt16(w, t.ErrorCode) +} + +// deleteTopics deletes the specified topics. +// +// See http://kafka.apache.org/protocol.html#The_Messages_DeleteTopics +func (c *Conn) deleteTopics(request deleteTopicsRequestV1) (deleteTopicsResponseV1, error) { + var response deleteTopicsResponseV1 + err := c.writeOperation( + func(deadline time.Time, id int32) error { + if request.Timeout == 0 { + now := time.Now() + deadline = adjustDeadlineForRTT(deadline, now, defaultRTT) + request.Timeout = milliseconds(deadlineToTimeout(deadline, now)) + } + return c.writeRequest(deleteTopicsRequest, v1, id, request) + }, + func(deadline time.Time, size int) error { + return expectZeroSize(func() (remain int, err error) { + return (&response).readFrom(&c.rbuf, size) + }()) + }, + ) + if err != nil { + return deleteTopicsResponseV1{}, err + } + for _, c := range response.TopicErrorCodes { + if c.ErrorCode != 0 { + return response, Error(c.ErrorCode) + } + } + return response, nil +} diff --git a/vendor/github.com/segmentio/kafka-go/describegroups.go b/vendor/github.com/segmentio/kafka-go/describegroups.go new file mode 100644 index 000000000..a7b7982ac --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/describegroups.go @@ -0,0 +1,186 @@ +package kafka + +import "bufio" + +// See http://kafka.apache.org/protocol.html#The_Messages_DescribeGroups +type describeGroupsRequestV1 struct { + // List of groupIds to request metadata for (an empty groupId array + // will return empty group metadata). + GroupIDs []string +} + +func (t describeGroupsRequestV1) size() int32 { + return sizeofStringArray(t.GroupIDs) +} + +func (t describeGroupsRequestV1) writeTo(w *bufio.Writer) { + writeStringArray(w, t.GroupIDs) +} + +type describeGroupsResponseMemberV1 struct { + // MemberID assigned by the group coordinator + MemberID string + + // ClientID used in the member's latest join group request + ClientID string + + // ClientHost used in the request session corresponding to the member's + // join group. + ClientHost string + + // MemberMetadata the metadata corresponding to the current group protocol + // in use (will only be present if the group is stable). + MemberMetadata []byte + + // MemberAssignments provided by the group leader (will only be present if + // the group is stable). + // + // See consumer groups section of https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol + MemberAssignments []byte +} + +func (t describeGroupsResponseMemberV1) size() int32 { + return sizeofString(t.MemberID) + + sizeofString(t.ClientID) + + sizeofString(t.ClientHost) + + sizeofBytes(t.MemberMetadata) + + sizeofBytes(t.MemberAssignments) +} + +func (t describeGroupsResponseMemberV1) writeTo(w *bufio.Writer) { + writeString(w, t.MemberID) + writeString(w, t.ClientID) + writeString(w, t.ClientHost) + writeBytes(w, t.MemberMetadata) + writeBytes(w, t.MemberAssignments) +} + +func (t *describeGroupsResponseMemberV1) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readString(r, size, &t.MemberID); err != nil { + return + } + if remain, err = readString(r, remain, &t.ClientID); err != nil { + return + } + if remain, err = readString(r, remain, &t.ClientHost); err != nil { + return + } + if remain, err = readBytes(r, remain, &t.MemberMetadata); err != nil { + return + } + if remain, err = readBytes(r, remain, &t.MemberAssignments); err != nil { + return + } + return +} + +type describeGroupsResponseGroupV1 struct { + // ErrorCode holds response error code + ErrorCode int16 + + // GroupID holds the unique group identifier + GroupID string + + // State holds current state of the group (one of: Dead, Stable, AwaitingSync, + // PreparingRebalance, or empty if there is no active group) + State string + + // ProtocolType holds the current group protocol type (will be empty if there is + // no active group) + ProtocolType string + + // Protocol holds the current group protocol (only provided if the group is Stable) + Protocol string + + // Members contains the current group members (only provided if the group is not Dead) + Members []describeGroupsResponseMemberV1 +} + +func (t describeGroupsResponseGroupV1) size() int32 { + return sizeofInt16(t.ErrorCode) + + sizeofString(t.GroupID) + + sizeofString(t.State) + + sizeofString(t.ProtocolType) + + sizeofString(t.Protocol) + + sizeofArray(len(t.Members), func(i int) int32 { return t.Members[i].size() }) +} + +func (t describeGroupsResponseGroupV1) writeTo(w *bufio.Writer) { + writeInt16(w, t.ErrorCode) + writeString(w, t.GroupID) + writeString(w, t.State) + writeString(w, t.ProtocolType) + writeString(w, t.Protocol) + writeArray(w, len(t.Members), func(i int) { t.Members[i].writeTo(w) }) +} + +func (t *describeGroupsResponseGroupV1) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readInt16(r, size, &t.ErrorCode); err != nil { + return + } + if remain, err = readString(r, remain, &t.GroupID); err != nil { + return + } + if remain, err = readString(r, remain, &t.State); err != nil { + return + } + if remain, err = readString(r, remain, &t.ProtocolType); err != nil { + return + } + if remain, err = readString(r, remain, &t.Protocol); err != nil { + return + } + + fn := func(r *bufio.Reader, size int) (fnRemain int, fnErr error) { + item := describeGroupsResponseMemberV1{} + if fnRemain, fnErr = (&item).readFrom(r, size); err != nil { + return + } + t.Members = append(t.Members, item) + return + } + if remain, err = readArrayWith(r, remain, fn); err != nil { + return + } + + return +} + +type describeGroupsResponseV1 struct { + // Duration in milliseconds for which the request was throttled due + // to quota violation (Zero if the request did not violate any quota) + ThrottleTimeMS int32 + + // Groups holds selected group information + Groups []describeGroupsResponseGroupV1 +} + +func (t describeGroupsResponseV1) size() int32 { + return sizeofInt32(t.ThrottleTimeMS) + + sizeofArray(len(t.Groups), func(i int) int32 { return t.Groups[i].size() }) +} + +func (t describeGroupsResponseV1) writeTo(w *bufio.Writer) { + writeInt32(w, t.ThrottleTimeMS) + writeArray(w, len(t.Groups), func(i int) { t.Groups[i].writeTo(w) }) +} + +func (t *describeGroupsResponseV1) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readInt32(r, size, &t.ThrottleTimeMS); err != nil { + return + } + + fn := func(r *bufio.Reader, size int) (fnRemain int, fnErr error) { + item := describeGroupsResponseGroupV1{} + if fnRemain, fnErr = (&item).readFrom(r, size); fnErr != nil { + return + } + t.Groups = append(t.Groups, item) + return + } + if remain, err = readArrayWith(r, remain, fn); err != nil { + return + } + + return +} diff --git a/vendor/github.com/segmentio/kafka-go/dialer.go b/vendor/github.com/segmentio/kafka-go/dialer.go new file mode 100644 index 000000000..57da750de --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/dialer.go @@ -0,0 +1,324 @@ +package kafka + +import ( + "context" + "crypto/tls" + "net" + "strconv" + "time" +) + +// The Dialer type mirrors the net.Dialer API but is designed to open kafka +// connections instead of raw network connections. +type Dialer struct { + // Unique identifier for client connections established by this Dialer. + ClientID string + + // Timeout is the maximum amount of time a dial will wait for a connect to + // complete. If Deadline is also set, it may fail earlier. + // + // The default is no timeout. + // + // When dialing a name with multiple IP addresses, the timeout may be + // divided between them. + // + // With or without a timeout, the operating system may impose its own + // earlier timeout. For instance, TCP timeouts are often around 3 minutes. + Timeout time.Duration + + // Deadline is the absolute point in time after which dials will fail. + // If Timeout is set, it may fail earlier. + // Zero means no deadline, or dependent on the operating system as with the + // Timeout option. + Deadline time.Time + + // LocalAddr is the local address to use when dialing an address. + // The address must be of a compatible type for the network being dialed. + // If nil, a local address is automatically chosen. + LocalAddr net.Addr + + // DualStack enables RFC 6555-compliant "Happy Eyeballs" dialing when the + // network is "tcp" and the destination is a host name with both IPv4 and + // IPv6 addresses. This allows a client to tolerate networks where one + // address family is silently broken. + DualStack bool + + // FallbackDelay specifies the length of time to wait before spawning a + // fallback connection, when DualStack is enabled. + // If zero, a default delay of 300ms is used. + FallbackDelay time.Duration + + // KeepAlive specifies the keep-alive period for an active network + // connection. + // If zero, keep-alives are not enabled. Network protocols that do not + // support keep-alives ignore this field. + KeepAlive time.Duration + + // Resolver optionally specifies an alternate resolver to use. + Resolver Resolver + + // TLS enables Dialer to open secure connections. If nil, standard net.Conn + // will be used. + TLS *tls.Config +} + +// Dial connects to the address on the named network. +func (d *Dialer) Dial(network string, address string) (*Conn, error) { + return d.DialContext(context.Background(), network, address) +} + +// DialContext connects to the address on the named network using the provided +// context. +// +// The provided Context must be non-nil. If the context expires before the +// connection is complete, an error is returned. Once successfully connected, +// any expiration of the context will not affect the connection. +// +// When using TCP, and the host in the address parameter resolves to multiple +// network addresses, any dial timeout (from d.Timeout or ctx) is spread over +// each consecutive dial, such that each is given an appropriate fraction of the +// time to connect. For example, if a host has 4 IP addresses and the timeout is +// 1 minute, the connect to each single address will be given 15 seconds to +// complete before trying the next one. +func (d *Dialer) DialContext(ctx context.Context, network string, address string) (*Conn, error) { + if d.Timeout != 0 { + var cancel context.CancelFunc + ctx, cancel = context.WithTimeout(ctx, d.Timeout) + defer cancel() + } + + if !d.Deadline.IsZero() { + var cancel context.CancelFunc + ctx, cancel = context.WithDeadline(ctx, d.Deadline) + defer cancel() + } + + c, err := d.dialContext(ctx, network, address) + if err != nil { + return nil, err + } + return NewConnWith(c, ConnConfig{ClientID: d.ClientID}), nil +} + +// DialLeader opens a connection to the leader of the partition for a given +// topic. +// +// The address given to the DialContext method may not be the one that the +// connection will end up being established to, because the dialer will lookup +// the partition leader for the topic and return a connection to that server. +// The original address is only used as a mechanism to discover the +// configuration of the kafka cluster that we're connecting to. +func (d *Dialer) DialLeader(ctx context.Context, network string, address string, topic string, partition int) (*Conn, error) { + b, err := d.LookupLeader(ctx, network, address, topic, partition) + if err != nil { + return nil, err + } + + c, err := d.dialContext(ctx, network, net.JoinHostPort(b.Host, strconv.Itoa(b.Port))) + if err != nil { + return nil, err + } + + return NewConnWith(c, ConnConfig{ + ClientID: d.ClientID, + Topic: topic, + Partition: partition, + }), nil +} + +// LookupLeader searches for the kafka broker that is the leader of the +// partition for a given topic, returning a Broker value representing it. +func (d *Dialer) LookupLeader(ctx context.Context, network string, address string, topic string, partition int) (Broker, error) { + c, err := d.DialContext(ctx, network, address) + if err != nil { + return Broker{}, err + } + defer c.Close() + + brkch := make(chan Broker, 1) + errch := make(chan error, 1) + + go func() { + for attempt := 0; true; attempt++ { + if attempt != 0 { + sleep(ctx, backoff(attempt, 100*time.Millisecond, 10*time.Second)) + } + + partitions, err := c.ReadPartitions(topic) + if err != nil { + if isTemporary(err) { + continue + } + errch <- err + return + } + + for _, p := range partitions { + if p.ID == partition { + brkch <- p.Leader + return + } + } + } + + errch <- UnknownTopicOrPartition + }() + + var brk Broker + select { + case brk = <-brkch: + case err = <-errch: + case <-ctx.Done(): + err = ctx.Err() + } + return brk, err +} + +// LookupPartitions returns the list of partitions that exist for the given topic. +func (d *Dialer) LookupPartitions(ctx context.Context, network string, address string, topic string) ([]Partition, error) { + conn, err := d.DialContext(ctx, network, address) + if err != nil { + return nil, err + } + defer conn.Close() + + prtch := make(chan []Partition, 1) + errch := make(chan error, 1) + + go func() { + if prt, err := conn.ReadPartitions(topic); err != nil { + errch <- err + } else { + prtch <- prt + } + }() + + var prt []Partition + select { + case prt = <-prtch: + case err = <-errch: + case <-ctx.Done(): + err = ctx.Err() + } + return prt, err +} + +// connectTLS returns a tls.Conn that has already completed the Handshake +func (d *Dialer) connectTLS(ctx context.Context, conn net.Conn) (tlsConn *tls.Conn, err error) { + tlsConn = tls.Client(conn, d.TLS) + errch := make(chan error) + + go func() { + defer close(errch) + errch <- tlsConn.Handshake() + }() + + select { + case <-ctx.Done(): + conn.Close() + tlsConn.Close() + <-errch // ignore possible error from Handshake + err = ctx.Err() + + case err = <-errch: + } + + return +} + +func (d *Dialer) dialContext(ctx context.Context, network string, address string) (net.Conn, error) { + if r := d.Resolver; r != nil { + host, port := splitHostPort(address) + addrs, err := r.LookupHost(ctx, host) + if err != nil { + return nil, err + } + if len(addrs) != 0 { + address = addrs[0] + } + if len(port) != 0 { + address, _ = splitHostPort(address) + address = net.JoinHostPort(address, port) + } + } + + conn, err := (&net.Dialer{ + LocalAddr: d.LocalAddr, + DualStack: d.DualStack, + FallbackDelay: d.FallbackDelay, + KeepAlive: d.KeepAlive, + }).DialContext(ctx, network, address) + if err != nil { + return nil, err + } + + if d.TLS != nil { + return d.connectTLS(ctx, conn) + } + + return conn, nil +} + +// DefaultDialer is the default dialer used when none is specified. +var DefaultDialer = &Dialer{ + Timeout: 10 * time.Second, + DualStack: true, +} + +// Dial is a convenience wrapper for DefaultDialer.Dial. +func Dial(network string, address string) (*Conn, error) { + return DefaultDialer.Dial(network, address) +} + +// DialContext is a convenience wrapper for DefaultDialer.DialContext. +func DialContext(ctx context.Context, network string, address string) (*Conn, error) { + return DefaultDialer.DialContext(ctx, network, address) +} + +// DialLeader is a convenience wrapper for DefaultDialer.DialLeader. +func DialLeader(ctx context.Context, network string, address string, topic string, partition int) (*Conn, error) { + return DefaultDialer.DialLeader(ctx, network, address, topic, partition) +} + +// The Resolver interface is used as an abstraction to provide service discovery +// of the hosts of a kafka cluster. +type Resolver interface { + // LookupHost looks up the given host using the local resolver. + // It returns a slice of that host's addresses. + LookupHost(ctx context.Context, host string) (addrs []string, err error) +} + +func sleep(ctx context.Context, duration time.Duration) bool { + if duration == 0 { + select { + default: + return true + case <-ctx.Done(): + return false + } + } + timer := time.NewTimer(duration) + defer timer.Stop() + select { + case <-timer.C: + return true + case <-ctx.Done(): + return false + } +} + +func backoff(attempt int, min time.Duration, max time.Duration) time.Duration { + d := time.Duration(attempt*attempt) * min + if d > max { + d = max + } + return d +} + +func splitHostPort(s string) (host string, port string) { + host, port, _ = net.SplitHostPort(s) + if len(host) == 0 && len(port) == 0 { + host = s + } + return +} diff --git a/vendor/github.com/segmentio/kafka-go/discard.go b/vendor/github.com/segmentio/kafka-go/discard.go new file mode 100644 index 000000000..bbb56b620 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/discard.go @@ -0,0 +1,42 @@ +package kafka + +import "bufio" + +func discardN(r *bufio.Reader, sz int, n int) (int, error) { + n, err := r.Discard(n) + return sz - n, err +} + +func discardInt8(r *bufio.Reader, sz int) (int, error) { + return discardN(r, sz, 1) +} + +func discardInt16(r *bufio.Reader, sz int) (int, error) { + return discardN(r, sz, 2) +} + +func discardInt32(r *bufio.Reader, sz int) (int, error) { + return discardN(r, sz, 4) +} + +func discardInt64(r *bufio.Reader, sz int) (int, error) { + return discardN(r, sz, 8) +} + +func discardString(r *bufio.Reader, sz int) (int, error) { + return readStringWith(r, sz, func(r *bufio.Reader, sz int, n int) (int, error) { + if n < 0 { + return sz, nil + } + return discardN(r, sz, n) + }) +} + +func discardBytes(r *bufio.Reader, sz int) (int, error) { + return readBytesWith(r, sz, func(r *bufio.Reader, sz int, n int) (int, error) { + if n < 0 { + return sz, nil + } + return discardN(r, sz, n) + }) +} diff --git a/vendor/github.com/segmentio/kafka-go/error.go b/vendor/github.com/segmentio/kafka-go/error.go new file mode 100644 index 000000000..0559af628 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/error.go @@ -0,0 +1,360 @@ +package kafka + +import ( + "fmt" + "io" +) + +// Error represents the different error codes that may be returned by kafka. +type Error int + +const ( + Unknown Error = -1 + OffsetOutOfRange Error = 1 + InvalidMessage Error = 2 + UnknownTopicOrPartition Error = 3 + InvalidMessageSize Error = 4 + LeaderNotAvailable Error = 5 + NotLeaderForPartition Error = 6 + RequestTimedOut Error = 7 + BrokerNotAvailable Error = 8 + ReplicaNotAvailable Error = 9 + MessageSizeTooLarge Error = 10 + StaleControllerEpoch Error = 11 + OffsetMetadataTooLarge Error = 12 + GroupLoadInProgress Error = 14 + GroupCoordinatorNotAvailable Error = 15 + NotCoordinatorForGroup Error = 16 + InvalidTopic Error = 17 + RecordListTooLarge Error = 18 + NotEnoughReplicas Error = 19 + NotEnoughReplicasAfterAppend Error = 20 + InvalidRequiredAcks Error = 21 + IllegalGeneration Error = 22 + InconsistentGroupProtocol Error = 23 + InvalidGroupId Error = 24 + UnknownMemberId Error = 25 + InvalidSessionTimeout Error = 26 + RebalanceInProgress Error = 27 + InvalidCommitOffsetSize Error = 28 + TopicAuthorizationFailed Error = 29 + GroupAuthorizationFailed Error = 30 + ClusterAuthorizationFailed Error = 31 + InvalidTimestamp Error = 32 + UnsupportedSASLMechanism Error = 33 + IllegalSASLState Error = 34 + UnsupportedVersion Error = 35 + TopicAlreadyExists Error = 36 + InvalidPartitionNumber Error = 37 + InvalidReplicationFactor Error = 38 + InvalidReplicaAssignment Error = 39 + InvalidConfiguration Error = 40 + NotController Error = 41 + InvalidRequest Error = 42 + UnsupportedForMessageFormat Error = 43 + PolicyViolation Error = 44 + OutOfOrderSequenceNumber Error = 45 + DuplicateSequenceNumber Error = 46 + InvalidProducerEpoch Error = 47 + InvalidTransactionState Error = 48 + InvalidProducerIDMapping Error = 49 + InvalidTransactionTimeout Error = 50 + ConcurrentTransactions Error = 51 + TransactionCoordinatorFenced Error = 52 + TransactionalIDAuthorizationFailed Error = 53 + SecurityDisabled Error = 54 + BrokerAuthorizationFailed Error = 55 +) + +// Error satisfies the error interface. +func (e Error) Error() string { + return fmt.Sprintf("[%d] %s: %s", e, e.Title(), e.Description()) +} + +// Timeout returns true if the error was due to a timeout. +func (e Error) Timeout() bool { + return e == RequestTimedOut +} + +// Temporary returns true if the operation that generated the error may succeed +// if retried at a later time. +func (e Error) Temporary() bool { + return e == LeaderNotAvailable || + e == BrokerNotAvailable || + e == ReplicaNotAvailable || + e == GroupLoadInProgress || + e == GroupCoordinatorNotAvailable || + e == RebalanceInProgress || + e.Timeout() +} + +// Title returns a human readable title for the error. +func (e Error) Title() string { + switch e { + case Unknown: + return "Unknown" + case OffsetOutOfRange: + return "Offset Out Of Range" + case InvalidMessage: + return "Invalid Message" + case UnknownTopicOrPartition: + return "Unknown Topic Or Partition" + case InvalidMessageSize: + return "Invalid Message Size" + case LeaderNotAvailable: + return "Leader Not Available" + case NotLeaderForPartition: + return "Not Leader For Partition" + case RequestTimedOut: + return "Request Timed Out" + case BrokerNotAvailable: + return "Broker Not Available" + case ReplicaNotAvailable: + return "Replica Not Available" + case MessageSizeTooLarge: + return "Message Size Too Large" + case StaleControllerEpoch: + return "Stale Controller Epoch" + case OffsetMetadataTooLarge: + return "Offset Metadata Too Large" + case GroupLoadInProgress: + return "Group Load In Progress" + case GroupCoordinatorNotAvailable: + return "Group Coordinator Not Available" + case NotCoordinatorForGroup: + return "Not Coordinator For Group" + case InvalidTopic: + return "Invalid Topic" + case RecordListTooLarge: + return "Record List Too Large" + case NotEnoughReplicas: + return "Not Enough Replicas" + case NotEnoughReplicasAfterAppend: + return "Not Enough Replicas After Append" + case InvalidRequiredAcks: + return "Invalid Required Acks" + case IllegalGeneration: + return "Illegal Generation" + case InconsistentGroupProtocol: + return "Inconsistent Group Protocol" + case InvalidGroupId: + return "Invalid Group ID" + case UnknownMemberId: + return "Unknown Member ID" + case InvalidSessionTimeout: + return "Invalid Session Timeout" + case RebalanceInProgress: + return "Rebalance In Progress" + case InvalidCommitOffsetSize: + return "Invalid Commit Offset Size" + case TopicAuthorizationFailed: + return "Topic Authorization Failed" + case GroupAuthorizationFailed: + return "Group Authorization Failed" + case ClusterAuthorizationFailed: + return "Cluster Authorization Failed" + case InvalidTimestamp: + return "Invalid Timestamp" + case UnsupportedSASLMechanism: + return "Unsupported SASL Mechanism" + case IllegalSASLState: + return "Illegal SASL State" + case UnsupportedVersion: + return "Unsupported Version" + case TopicAlreadyExists: + return "Topic Already Exists" + case InvalidPartitionNumber: + return "Invalid Partition Number" + case InvalidReplicationFactor: + return "Invalid Replication Factor" + case InvalidReplicaAssignment: + return "Invalid Replica Assignment" + case InvalidConfiguration: + return "Invalid Configuration" + case NotController: + return "Not Controller" + case InvalidRequest: + return "Invalid Request" + case UnsupportedForMessageFormat: + return "Unsupported For Message Format" + case PolicyViolation: + return "Policy Violation" + case OutOfOrderSequenceNumber: + return "Out Of Order Sequence Number" + case DuplicateSequenceNumber: + return "Duplicate Sequence Number" + case InvalidProducerEpoch: + return "Invalid Producer Epoch" + case InvalidTransactionState: + return "Invalid Transaction State" + case InvalidProducerIDMapping: + return "Invalid Producer ID Mapping" + case InvalidTransactionTimeout: + return "Invalid Transaction Timeout" + case ConcurrentTransactions: + return "Concurrent Transactions" + case TransactionCoordinatorFenced: + return "Transaction Coordinator Fenced" + case TransactionalIDAuthorizationFailed: + return "Transactional ID Authorization Failed" + case SecurityDisabled: + return "Security Disabled" + case BrokerAuthorizationFailed: + return "Broker Authorization Failed" + } + return "" +} + +// Description returns a human readable description of cause of the error. +func (e Error) Description() string { + switch e { + case Unknown: + return "an unexpected server error occurred" + case OffsetOutOfRange: + return "the requested offset is outside the range of offsets maintained by the server for the given topic/partition" + case InvalidMessage: + return "the message contents does not match its CRC" + case UnknownTopicOrPartition: + return "the request is for a topic or partition that does not exist on this broker" + case InvalidMessageSize: + return "the message has a negative size" + case LeaderNotAvailable: + return "the cluster is in the middle of a leadership election and there is currently no leader for this partition and hence it is unavailable for writes" + case NotLeaderForPartition: + return "the client attempted to send messages to a replica that is not the leader for some partition, the client's metadata are likely out of date" + case RequestTimedOut: + return "the request exceeded the user-specified time limit in the request" + case BrokerNotAvailable: + return "not a client facing error and is used mostly by tools when a broker is not alive" + case ReplicaNotAvailable: + return "a replica is expected on a broker, but is not (this can be safely ignored)" + case MessageSizeTooLarge: + return "the server has a configurable maximum message size to avoid unbounded memory allocation and the client attempted to produce a message larger than this maximum" + case StaleControllerEpoch: + return "internal error code for broker-to-broker communication" + case OffsetMetadataTooLarge: + return "the client specified a string larger than configured maximum for offset metadata" + case GroupLoadInProgress: + return "the broker returns this error code for an offset fetch request if it is still loading offsets (after a leader change for that offsets topic partition), or in response to group membership requests (such as heartbeats) when group metadata is being loaded by the coordinator" + case GroupCoordinatorNotAvailable: + return "the broker returns this error code for group coordinator requests, offset commits, and most group management requests if the offsets topic has not yet been created, or if the group coordinator is not active" + case NotCoordinatorForGroup: + return "the broker returns this error code if it receives an offset fetch or commit request for a group that it is not a coordinator for" + case InvalidTopic: + return "a request which attempted to access an invalid topic (e.g. one which has an illegal name), or if an attempt was made to write to an internal topic (such as the consumer offsets topic)" + case RecordListTooLarge: + return "a message batch in a produce request exceeds the maximum configured segment size" + case NotEnoughReplicas: + return "the number of in-sync replicas is lower than the configured minimum and requiredAcks is -1" + case NotEnoughReplicasAfterAppend: + return "the message was written to the log, but with fewer in-sync replicas than required." + case InvalidRequiredAcks: + return "the requested requiredAcks is invalid (anything other than -1, 1, or 0)" + case IllegalGeneration: + return "the generation id provided in the request is not the current generation" + case InconsistentGroupProtocol: + return "the member provided a protocol type or set of protocols which is not compatible with the current group" + case InvalidGroupId: + return "the group id is empty or null" + case UnknownMemberId: + return "the member id is not in the current generation" + case InvalidSessionTimeout: + return "the requested session timeout is outside of the allowed range on the broker" + case RebalanceInProgress: + return "the coordinator has begun rebalancing the group, the client should rejoin the group" + case InvalidCommitOffsetSize: + return "an offset commit was rejected because of oversize metadata" + case TopicAuthorizationFailed: + return "the client is not authorized to access the requested topic" + case GroupAuthorizationFailed: + return "the client is not authorized to access a particular group id" + case ClusterAuthorizationFailed: + return "the client is not authorized to use an inter-broker or administrative API" + case InvalidTimestamp: + return "the timestamp of the message is out of acceptable range" + case UnsupportedSASLMechanism: + return "the broker does not support the requested SASL mechanism" + case IllegalSASLState: + return "the request is not valid given the current SASL state" + case UnsupportedVersion: + return "the version of API is not supported" + case TopicAlreadyExists: + return "a topic with this name already exists" + case InvalidPartitionNumber: + return "the number of partitions is invalid" + case InvalidReplicationFactor: + return "the replication-factor is invalid" + case InvalidReplicaAssignment: + return "the replica assignment is invalid" + case InvalidConfiguration: + return "the configuration is invalid" + case NotController: + return "this is not the correct controller for this cluster" + case InvalidRequest: + return "this most likely occurs because of a request being malformed by the client library or the message was sent to an incompatible broker, se the broker logs for more details" + case UnsupportedForMessageFormat: + return "the message format version on the broker does not support the request" + case PolicyViolation: + return "the request parameters do not satisfy the configured policy" + case OutOfOrderSequenceNumber: + return "the broker received an out of order sequence number" + case DuplicateSequenceNumber: + return "the broker received a duplicate sequence number" + case InvalidProducerEpoch: + return "the producer attempted an operation with an old epoch, either there is a newer producer with the same transactional ID, or the producer's transaction has been expired by the broker" + case InvalidTransactionState: + return "the producer attempted a transactional operation in an invalid state" + case InvalidProducerIDMapping: + return "the producer attempted to use a producer id which is not currently assigned to its transactional ID" + case InvalidTransactionTimeout: + return "the transaction timeout is larger than the maximum value allowed by the broker (as configured by max.transaction.timeout.ms)" + case ConcurrentTransactions: + return "the producer attempted to update a transaction while another concurrent operation on the same transaction was ongoing" + case TransactionCoordinatorFenced: + return "the transaction coordinator sending a WriteTxnMarker is no longer the current coordinator for a given producer" + case TransactionalIDAuthorizationFailed: + return "the transactional ID authorization failed" + case SecurityDisabled: + return "the security features are disabled" + case BrokerAuthorizationFailed: + return "the broker authorization failed" + } + return "" +} + +func isTimeout(err error) bool { + e, ok := err.(interface { + Timeout() bool + }) + return ok && e.Timeout() +} + +func isTemporary(err error) bool { + e, ok := err.(interface { + Temporary() bool + }) + return ok && e.Temporary() +} + +func silentEOF(err error) error { + if err == io.EOF { + err = nil + } + return err +} + +func dontExpectEOF(err error) error { + if err == io.EOF { + err = io.ErrUnexpectedEOF + } + return err +} + +func coalesceErrors(errs ...error) error { + for _, err := range errs { + if err != nil { + return err + } + } + return nil +} diff --git a/vendor/github.com/segmentio/kafka-go/fetch.go b/vendor/github.com/segmentio/kafka-go/fetch.go new file mode 100644 index 000000000..326594dcc --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/fetch.go @@ -0,0 +1,101 @@ +package kafka + +import "bufio" + +type fetchRequestV1 struct { + ReplicaID int32 + MaxWaitTime int32 + MinBytes int32 + Topics []fetchRequestTopicV1 +} + +func (r fetchRequestV1) size() int32 { + return 4 + 4 + 4 + sizeofArray(len(r.Topics), func(i int) int32 { return r.Topics[i].size() }) +} + +func (r fetchRequestV1) writeTo(w *bufio.Writer) { + writeInt32(w, r.ReplicaID) + writeInt32(w, r.MaxWaitTime) + writeInt32(w, r.MinBytes) + writeArray(w, len(r.Topics), func(i int) { r.Topics[i].writeTo(w) }) +} + +type fetchRequestTopicV1 struct { + TopicName string + Partitions []fetchRequestPartitionV1 +} + +func (t fetchRequestTopicV1) size() int32 { + return sizeofString(t.TopicName) + + sizeofArray(len(t.Partitions), func(i int) int32 { return t.Partitions[i].size() }) +} + +func (t fetchRequestTopicV1) writeTo(w *bufio.Writer) { + writeString(w, t.TopicName) + writeArray(w, len(t.Partitions), func(i int) { t.Partitions[i].writeTo(w) }) +} + +type fetchRequestPartitionV1 struct { + Partition int32 + FetchOffset int64 + MaxBytes int32 +} + +func (p fetchRequestPartitionV1) size() int32 { + return 4 + 8 + 4 +} + +func (p fetchRequestPartitionV1) writeTo(w *bufio.Writer) { + writeInt32(w, p.Partition) + writeInt64(w, p.FetchOffset) + writeInt32(w, p.MaxBytes) +} + +type fetchResponseV1 struct { + ThrottleTime int32 + Topics []fetchResponseTopicV1 +} + +func (r fetchResponseV1) size() int32 { + return 4 + sizeofArray(len(r.Topics), func(i int) int32 { return r.Topics[i].size() }) +} + +func (r fetchResponseV1) writeTo(w *bufio.Writer) { + writeInt32(w, r.ThrottleTime) + writeArray(w, len(r.Topics), func(i int) { r.Topics[i].writeTo(w) }) +} + +type fetchResponseTopicV1 struct { + TopicName string + Partitions []fetchResponsePartitionV1 +} + +func (t fetchResponseTopicV1) size() int32 { + return sizeofString(t.TopicName) + + sizeofArray(len(t.Partitions), func(i int) int32 { return t.Partitions[i].size() }) +} + +func (t fetchResponseTopicV1) writeTo(w *bufio.Writer) { + writeString(w, t.TopicName) + writeArray(w, len(t.Partitions), func(i int) { t.Partitions[i].writeTo(w) }) +} + +type fetchResponsePartitionV1 struct { + Partition int32 + ErrorCode int16 + HighwaterMarkOffset int64 + MessageSetSize int32 + MessageSet messageSet +} + +func (p fetchResponsePartitionV1) size() int32 { + return 4 + 2 + 8 + 4 + p.MessageSet.size() +} + +func (p fetchResponsePartitionV1) writeTo(w *bufio.Writer) { + writeInt32(w, p.Partition) + writeInt16(w, p.ErrorCode) + writeInt64(w, p.HighwaterMarkOffset) + writeInt32(w, p.MessageSetSize) + p.MessageSet.writeTo(w) +} diff --git a/vendor/github.com/segmentio/kafka-go/findcoordinator.go b/vendor/github.com/segmentio/kafka-go/findcoordinator.go new file mode 100644 index 000000000..750ba7c87 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/findcoordinator.go @@ -0,0 +1,108 @@ +package kafka + +import ( + "bufio" +) + +// FindCoordinatorRequestV1 requests the coordinator for the specified group or transaction +// +// See http://kafka.apache.org/protocol.html#The_Messages_FindCoordinator +type findCoordinatorRequestV1 struct { + // CoordinatorKey holds id to use for finding the coordinator (for groups, this is + // the groupId, for transactional producers, this is the transactional id) + CoordinatorKey string + + // CoordinatorType indicates type of coordinator to find (0 = group, 1 = transaction) + CoordinatorType int8 +} + +func (t findCoordinatorRequestV1) size() int32 { + return sizeofString(t.CoordinatorKey) + sizeof(t.CoordinatorType) +} + +func (t findCoordinatorRequestV1) writeTo(w *bufio.Writer) { + writeString(w, t.CoordinatorKey) + writeInt8(w, t.CoordinatorType) +} + +type findCoordinatorResponseCoordinatorV1 struct { + // NodeID holds the broker id. + NodeID int32 + + // Host of the broker + Host string + + // Port on which broker accepts requests + Port int32 +} + +func (t findCoordinatorResponseCoordinatorV1) size() int32 { + return sizeofInt32(t.NodeID) + + sizeofString(t.Host) + + sizeofInt32(t.Port) +} + +func (t findCoordinatorResponseCoordinatorV1) writeTo(w *bufio.Writer) { + writeInt32(w, t.NodeID) + writeString(w, t.Host) + writeInt32(w, t.Port) +} + +func (t *findCoordinatorResponseCoordinatorV1) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readInt32(r, size, &t.NodeID); err != nil { + return + } + if remain, err = readString(r, remain, &t.Host); err != nil { + return + } + if remain, err = readInt32(r, remain, &t.Port); err != nil { + return + } + return +} + +type findCoordinatorResponseV1 struct { + // ThrottleTimeMS holds the duration in milliseconds for which the request + // was throttled due to quota violation (Zero if the request did not violate + // any quota) + ThrottleTimeMS int32 + + // ErrorCode holds response error code + ErrorCode int16 + + // ErrorMessage holds response error message + ErrorMessage string + + // Coordinator holds host and port information for the coordinator + Coordinator findCoordinatorResponseCoordinatorV1 +} + +func (t findCoordinatorResponseV1) size() int32 { + return sizeofInt32(t.ThrottleTimeMS) + + sizeofInt16(t.ErrorCode) + + sizeofString(t.ErrorMessage) + + t.Coordinator.size() +} + +func (t findCoordinatorResponseV1) writeTo(w *bufio.Writer) { + writeInt32(w, t.ThrottleTimeMS) + writeInt16(w, t.ErrorCode) + writeString(w, t.ErrorMessage) + t.Coordinator.writeTo(w) +} + +func (t *findCoordinatorResponseV1) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readInt32(r, size, &t.ThrottleTimeMS); err != nil { + return + } + if remain, err = readInt16(r, remain, &t.ErrorCode); err != nil { + return + } + if remain, err = readString(r, remain, &t.ErrorMessage); err != nil { + return + } + if remain, err = (&t.Coordinator).readFrom(r, remain); err != nil { + return + } + return +} diff --git a/vendor/github.com/segmentio/kafka-go/heartbeat.go b/vendor/github.com/segmentio/kafka-go/heartbeat.go new file mode 100644 index 000000000..5a683adcf --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/heartbeat.go @@ -0,0 +1,56 @@ +package kafka + +import "bufio" + +type heartbeatRequestV1 struct { + // GroupID holds the unique group identifier + GroupID string + + // GenerationID holds the generation of the group. + GenerationID int32 + + // MemberID assigned by the group coordinator + MemberID string +} + +func (t heartbeatRequestV1) size() int32 { + return sizeofString(t.GroupID) + + sizeofInt32(t.GenerationID) + + sizeofString(t.MemberID) +} + +func (t heartbeatRequestV1) writeTo(w *bufio.Writer) { + writeString(w, t.GroupID) + writeInt32(w, t.GenerationID) + writeString(w, t.MemberID) +} + +type heartbeatResponseV1 struct { + // ThrottleTimeMS holds the duration in milliseconds for which the request + // was throttled due to quota violation (Zero if the request did not violate + // any quota) + ThrottleTimeMS int32 + + // ErrorCode holds response error code + ErrorCode int16 +} + +func (t heartbeatResponseV1) size() int32 { + return sizeofInt32(t.ThrottleTimeMS) + + sizeofInt16(t.ErrorCode) +} + +func (t heartbeatResponseV1) writeTo(w *bufio.Writer) { + writeInt32(w, t.ThrottleTimeMS) + writeInt16(w, t.ErrorCode) +} + +func (t *heartbeatResponseV1) readFrom(r *bufio.Reader, sz int) (remain int, err error) { + if remain, err = readInt32(r, sz, &t.ThrottleTimeMS); err != nil { + return + } + if remain, err = readInt16(r, remain, &t.ErrorCode); err != nil { + return + } + return +} diff --git a/vendor/github.com/segmentio/kafka-go/joingroup.go b/vendor/github.com/segmentio/kafka-go/joingroup.go new file mode 100644 index 000000000..2e1e06006 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/joingroup.go @@ -0,0 +1,212 @@ +package kafka + +import ( + "bufio" + "bytes" +) + +type memberGroupMetadata struct { + // MemberID assigned by the group coordinator or null if joining for the + // first time. + MemberID string + Metadata groupMetadata +} + +type groupMetadata struct { + Version int16 + Topics []string + UserData []byte +} + +func (t groupMetadata) size() int32 { + return sizeofInt16(t.Version) + + sizeofStringArray(t.Topics) + + sizeofBytes(t.UserData) +} + +func (t groupMetadata) writeTo(w *bufio.Writer) { + writeInt16(w, t.Version) + writeStringArray(w, t.Topics) + writeBytes(w, t.UserData) +} + +func (t groupMetadata) bytes() []byte { + buf := bytes.NewBuffer(nil) + w := bufio.NewWriter(buf) + t.writeTo(w) + w.Flush() + return buf.Bytes() +} + +func (t *groupMetadata) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readInt16(r, size, &t.Version); err != nil { + return + } + if remain, err = readStringArray(r, remain, &t.Topics); err != nil { + return + } + if remain, err = readBytes(r, remain, &t.UserData); err != nil { + return + } + return +} + +type joinGroupRequestGroupProtocolV2 struct { + ProtocolName string + ProtocolMetadata []byte +} + +func (t joinGroupRequestGroupProtocolV2) size() int32 { + return sizeofString(t.ProtocolName) + + sizeofBytes(t.ProtocolMetadata) +} + +func (t joinGroupRequestGroupProtocolV2) writeTo(w *bufio.Writer) { + writeString(w, t.ProtocolName) + writeBytes(w, t.ProtocolMetadata) +} + +type joinGroupRequestV2 struct { + // GroupID holds the unique group identifier + GroupID string + + // SessionTimeout holds the coordinator considers the consumer dead if it + // receives no heartbeat after this timeout in ms. + SessionTimeout int32 + + // RebalanceTimeout holds the maximum time that the coordinator will wait + // for each member to rejoin when rebalancing the group in ms + RebalanceTimeout int32 + + // MemberID assigned by the group coordinator or the zero string if joining + // for the first time. + MemberID string + + // ProtocolType holds the unique name for class of protocols implemented by group + ProtocolType string + + // GroupProtocols holds the list of protocols that the member supports + GroupProtocols []joinGroupRequestGroupProtocolV2 +} + +func (t joinGroupRequestV2) size() int32 { + return sizeofString(t.GroupID) + + sizeofInt32(t.SessionTimeout) + + sizeofInt32(t.RebalanceTimeout) + + sizeofString(t.MemberID) + + sizeofString(t.ProtocolType) + + sizeofArray(len(t.GroupProtocols), func(i int) int32 { return t.GroupProtocols[i].size() }) +} + +func (t joinGroupRequestV2) writeTo(w *bufio.Writer) { + writeString(w, t.GroupID) + writeInt32(w, t.SessionTimeout) + writeInt32(w, t.RebalanceTimeout) + writeString(w, t.MemberID) + writeString(w, t.ProtocolType) + writeArray(w, len(t.GroupProtocols), func(i int) { t.GroupProtocols[i].writeTo(w) }) +} + +type joinGroupResponseMemberV2 struct { + // MemberID assigned by the group coordinator + MemberID string + MemberMetadata []byte +} + +func (t joinGroupResponseMemberV2) size() int32 { + return sizeofString(t.MemberID) + + sizeofBytes(t.MemberMetadata) +} + +func (t joinGroupResponseMemberV2) writeTo(w *bufio.Writer) { + writeString(w, t.MemberID) + writeBytes(w, t.MemberMetadata) +} + +func (t *joinGroupResponseMemberV2) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readString(r, size, &t.MemberID); err != nil { + return + } + if remain, err = readBytes(r, remain, &t.MemberMetadata); err != nil { + return + } + return +} + +type joinGroupResponseV2 struct { + // ThrottleTimeMS holds the duration in milliseconds for which the request + // was throttled due to quota violation (Zero if the request did not violate + // any quota) + ThrottleTimeMS int32 + + // ErrorCode holds response error code + ErrorCode int16 + + // GenerationID holds the generation of the group. + GenerationID int32 + + // GroupProtocol holds the group protocol selected by the coordinator + GroupProtocol string + + // LeaderID holds the leader of the group + LeaderID string + + // MemberID assigned by the group coordinator + MemberID string + Members []joinGroupResponseMemberV2 +} + +func (t joinGroupResponseV2) size() int32 { + return sizeofInt32(t.ThrottleTimeMS) + + sizeofInt16(t.ErrorCode) + + sizeofInt32(t.GenerationID) + + sizeofString(t.GroupProtocol) + + sizeofString(t.LeaderID) + + sizeofString(t.MemberID) + + sizeofArray(len(t.MemberID), func(i int) int32 { return t.Members[i].size() }) +} + +func (t joinGroupResponseV2) writeTo(w *bufio.Writer) { + writeInt32(w, t.ThrottleTimeMS) + writeInt16(w, t.ErrorCode) + writeInt32(w, t.GenerationID) + writeString(w, t.GroupProtocol) + writeString(w, t.LeaderID) + writeString(w, t.MemberID) + writeArray(w, len(t.Members), func(i int) { t.Members[i].writeTo(w) }) +} + +func (t *joinGroupResponseV2) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readInt32(r, size, &t.ThrottleTimeMS); err != nil { + return + } + if remain, err = readInt16(r, remain, &t.ErrorCode); err != nil { + return + } + if remain, err = readInt32(r, remain, &t.GenerationID); err != nil { + return + } + if remain, err = readString(r, remain, &t.GroupProtocol); err != nil { + return + } + if remain, err = readString(r, remain, &t.LeaderID); err != nil { + return + } + if remain, err = readString(r, remain, &t.MemberID); err != nil { + return + } + + fn := func(r *bufio.Reader, size int) (fnRemain int, fnErr error) { + var item joinGroupResponseMemberV2 + if fnRemain, fnErr = (&item).readFrom(r, size); fnErr != nil { + return + } + t.Members = append(t.Members, item) + return + } + if remain, err = readArrayWith(r, remain, fn); err != nil { + return + } + + return +} diff --git a/vendor/github.com/segmentio/kafka-go/leavegroup.go b/vendor/github.com/segmentio/kafka-go/leavegroup.go new file mode 100644 index 000000000..e3625d473 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/leavegroup.go @@ -0,0 +1,52 @@ +package kafka + +import "bufio" + +type leaveGroupRequestV1 struct { + // GroupID holds the unique group identifier + GroupID string + + // MemberID assigned by the group coordinator or the zero string if joining + // for the first time. + MemberID string +} + +func (t leaveGroupRequestV1) size() int32 { + return sizeofString(t.GroupID) + + sizeofString(t.MemberID) +} + +func (t leaveGroupRequestV1) writeTo(w *bufio.Writer) { + writeString(w, t.GroupID) + writeString(w, t.MemberID) +} + +type leaveGroupResponseV1 struct { + // ThrottleTimeMS holds the duration in milliseconds for which the request + // was throttled due to quota violation (Zero if the request did not violate + // any quota) + ThrottleTimeMS int32 + + // ErrorCode holds response error code + ErrorCode int16 +} + +func (t leaveGroupResponseV1) size() int32 { + return sizeofInt32(t.ThrottleTimeMS) + + sizeofInt16(t.ErrorCode) +} + +func (t leaveGroupResponseV1) writeTo(w *bufio.Writer) { + writeInt32(w, t.ThrottleTimeMS) + writeInt16(w, t.ErrorCode) +} + +func (t *leaveGroupResponseV1) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readInt32(r, size, &t.ThrottleTimeMS); err != nil { + return + } + if remain, err = readInt16(r, remain, &t.ErrorCode); err != nil { + return + } + return +} diff --git a/vendor/github.com/segmentio/kafka-go/listgroups.go b/vendor/github.com/segmentio/kafka-go/listgroups.go new file mode 100644 index 000000000..a369f7d09 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/listgroups.go @@ -0,0 +1,87 @@ +package kafka + +import ( + "bufio" +) + +type listGroupsRequestV1 struct { +} + +func (t listGroupsRequestV1) size() int32 { + return 0 +} + +func (t listGroupsRequestV1) writeTo(w *bufio.Writer) { +} + +type ListGroupsResponseGroupV1 struct { + // GroupID holds the unique group identifier + GroupID string + ProtocolType string +} + +func (t ListGroupsResponseGroupV1) size() int32 { + return sizeofString(t.GroupID) + + sizeofString(t.ProtocolType) +} + +func (t ListGroupsResponseGroupV1) writeTo(w *bufio.Writer) { + writeString(w, t.GroupID) + writeString(w, t.ProtocolType) +} + +func (t *ListGroupsResponseGroupV1) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readString(r, size, &t.GroupID); err != nil { + return + } + if remain, err = readString(r, remain, &t.ProtocolType); err != nil { + return + } + return +} + +type listGroupsResponseV1 struct { + // ThrottleTimeMS holds the duration in milliseconds for which the request + // was throttled due to quota violation (Zero if the request did not violate + // any quota) + ThrottleTimeMS int32 + + // ErrorCode holds response error code + ErrorCode int16 + Groups []ListGroupsResponseGroupV1 +} + +func (t listGroupsResponseV1) size() int32 { + return sizeofInt32(t.ThrottleTimeMS) + + sizeofInt16(t.ErrorCode) + + sizeofArray(len(t.Groups), func(i int) int32 { return t.Groups[i].size() }) +} + +func (t listGroupsResponseV1) writeTo(w *bufio.Writer) { + writeInt32(w, t.ThrottleTimeMS) + writeInt16(w, t.ErrorCode) + writeArray(w, len(t.Groups), func(i int) { t.Groups[i].writeTo(w) }) +} + +func (t *listGroupsResponseV1) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readInt32(r, size, &t.ThrottleTimeMS); err != nil { + return + } + if remain, err = readInt16(r, remain, &t.ErrorCode); err != nil { + return + } + + fn := func(withReader *bufio.Reader, withSize int) (fnRemain int, fnErr error) { + var item ListGroupsResponseGroupV1 + if fnRemain, fnErr = (&item).readFrom(withReader, withSize); err != nil { + return + } + t.Groups = append(t.Groups, item) + return + } + if remain, err = readArrayWith(r, remain, fn); err != nil { + return + } + + return +} diff --git a/vendor/github.com/segmentio/kafka-go/listoffset.go b/vendor/github.com/segmentio/kafka-go/listoffset.go new file mode 100644 index 000000000..12903197c --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/listoffset.go @@ -0,0 +1,105 @@ +package kafka + +import "bufio" + +type listOffsetRequestV1 struct { + ReplicaID int32 + Topics []listOffsetRequestTopicV1 +} + +func (r listOffsetRequestV1) size() int32 { + return 4 + sizeofArray(len(r.Topics), func(i int) int32 { return r.Topics[i].size() }) +} + +func (r listOffsetRequestV1) writeTo(w *bufio.Writer) { + writeInt32(w, r.ReplicaID) + writeArray(w, len(r.Topics), func(i int) { r.Topics[i].writeTo(w) }) +} + +type listOffsetRequestTopicV1 struct { + TopicName string + Partitions []listOffsetRequestPartitionV1 +} + +func (t listOffsetRequestTopicV1) size() int32 { + return sizeofString(t.TopicName) + + sizeofArray(len(t.Partitions), func(i int) int32 { return t.Partitions[i].size() }) +} + +func (t listOffsetRequestTopicV1) writeTo(w *bufio.Writer) { + writeString(w, t.TopicName) + writeArray(w, len(t.Partitions), func(i int) { t.Partitions[i].writeTo(w) }) +} + +type listOffsetRequestPartitionV1 struct { + Partition int32 + Time int64 +} + +func (p listOffsetRequestPartitionV1) size() int32 { + return 4 + 8 +} + +func (p listOffsetRequestPartitionV1) writeTo(w *bufio.Writer) { + writeInt32(w, p.Partition) + writeInt64(w, p.Time) +} + +type listOffsetResponseV1 []listOffsetResponseTopicV1 + +func (r listOffsetResponseV1) size() int32 { + return sizeofArray(len(r), func(i int) int32 { return r[i].size() }) +} + +func (r listOffsetResponseV1) writeTo(w *bufio.Writer) { + writeArray(w, len(r), func(i int) { r[i].writeTo(w) }) +} + +type listOffsetResponseTopicV1 struct { + TopicName string + PartitionOffsets []partitionOffsetV1 +} + +func (t listOffsetResponseTopicV1) size() int32 { + return sizeofString(t.TopicName) + + sizeofArray(len(t.PartitionOffsets), func(i int) int32 { return t.PartitionOffsets[i].size() }) +} + +func (t listOffsetResponseTopicV1) writeTo(w *bufio.Writer) { + writeString(w, t.TopicName) + writeArray(w, len(t.PartitionOffsets), func(i int) { t.PartitionOffsets[i].writeTo(w) }) +} + +type partitionOffsetV1 struct { + Partition int32 + ErrorCode int16 + Timestamp int64 + Offset int64 +} + +func (p partitionOffsetV1) size() int32 { + return 4 + 2 + 8 + 8 +} + +func (p partitionOffsetV1) writeTo(w *bufio.Writer) { + writeInt32(w, p.Partition) + writeInt16(w, p.ErrorCode) + writeInt64(w, p.Timestamp) + writeInt64(w, p.Offset) +} + +func (p *partitionOffsetV1) readFrom(r *bufio.Reader, sz int) (remain int, err error) { + if remain, err = readInt32(r, sz, &p.Partition); err != nil { + return + } + if remain, err = readInt16(r, remain, &p.ErrorCode); err != nil { + return + } + if remain, err = readInt64(r, remain, &p.Timestamp); err != nil { + return + } + if remain, err = readInt64(r, remain, &p.Offset); err != nil { + return + } + return +} diff --git a/vendor/github.com/segmentio/kafka-go/message.go b/vendor/github.com/segmentio/kafka-go/message.go new file mode 100644 index 000000000..96e9afbee --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/message.go @@ -0,0 +1,105 @@ +package kafka + +import ( + "bufio" + "time" +) + +// Message is a data structure representing kafka messages. +type Message struct { + // Topic is reads only and MUST NOT be set when writing messages + Topic string + + // Partition is reads only and MUST NOT be set when writing messages + Partition int + Offset int64 + Key []byte + Value []byte + + // If not set at the creation, Time will be automatically set when + // writing the message. + Time time.Time +} + +func (msg Message) item() messageSetItem { + item := messageSetItem{ + Offset: msg.Offset, + Message: msg.message(), + } + item.MessageSize = item.Message.size() + return item +} + +func (msg Message) message() message { + m := message{ + MagicByte: 1, + Key: msg.Key, + Value: msg.Value, + Timestamp: timestamp(msg.Time), + } + m.CRC = m.crc32() + return m +} + +type message struct { + CRC int32 + MagicByte int8 + Attributes int8 + Timestamp int64 + Key []byte + Value []byte +} + +func (m message) crc32() int32 { + return int32(crc32OfMessage(m.MagicByte, m.Attributes, m.Timestamp, m.Key, m.Value)) +} + +func (m message) size() int32 { + size := 4 + 1 + 1 + sizeofBytes(m.Key) + sizeofBytes(m.Value) + if m.MagicByte != 0 { + size += 8 // Timestamp + } + return size +} + +func (m message) writeTo(w *bufio.Writer) { + writeInt32(w, m.CRC) + writeInt8(w, m.MagicByte) + writeInt8(w, m.Attributes) + if m.MagicByte != 0 { + writeInt64(w, m.Timestamp) + } + writeBytes(w, m.Key) + writeBytes(w, m.Value) +} + +type messageSetItem struct { + Offset int64 + MessageSize int32 + Message message +} + +func (m messageSetItem) size() int32 { + return 8 + 4 + m.Message.size() +} + +func (m messageSetItem) writeTo(w *bufio.Writer) { + writeInt64(w, m.Offset) + writeInt32(w, m.MessageSize) + m.Message.writeTo(w) +} + +type messageSet []messageSetItem + +func (s messageSet) size() (size int32) { + for _, m := range s { + size += m.size() + } + return +} + +func (s messageSet) writeTo(w *bufio.Writer) { + for _, m := range s { + m.writeTo(w) + } +} diff --git a/vendor/github.com/segmentio/kafka-go/metadata.go b/vendor/github.com/segmentio/kafka-go/metadata.go new file mode 100644 index 000000000..51fe98b22 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/metadata.go @@ -0,0 +1,83 @@ +package kafka + +import "bufio" + +type topicMetadataRequestV0 []string + +func (r topicMetadataRequestV0) size() int32 { + return sizeofStringArray([]string(r)) +} + +func (r topicMetadataRequestV0) writeTo(w *bufio.Writer) { + writeStringArray(w, []string(r)) +} + +type metadataResponseV0 struct { + Brokers []brokerMetadataV0 + Topics []topicMetadataV0 +} + +func (r metadataResponseV0) size() int32 { + n1 := sizeofArray(len(r.Brokers), func(i int) int32 { return r.Brokers[i].size() }) + n2 := sizeofArray(len(r.Topics), func(i int) int32 { return r.Topics[i].size() }) + return n1 + n2 +} + +func (r metadataResponseV0) writeTo(w *bufio.Writer) { + writeArray(w, len(r.Brokers), func(i int) { r.Brokers[i].writeTo(w) }) + writeArray(w, len(r.Topics), func(i int) { r.Topics[i].writeTo(w) }) +} + +type brokerMetadataV0 struct { + NodeID int32 + Host string + Port int32 +} + +func (b brokerMetadataV0) size() int32 { + return 4 + 4 + sizeofString(b.Host) +} + +func (b brokerMetadataV0) writeTo(w *bufio.Writer) { + writeInt32(w, b.NodeID) + writeString(w, b.Host) + writeInt32(w, b.Port) +} + +type topicMetadataV0 struct { + TopicErrorCode int16 + TopicName string + Partitions []partitionMetadataV0 +} + +func (t topicMetadataV0) size() int32 { + return 2 + + sizeofString(t.TopicName) + + sizeofArray(len(t.Partitions), func(i int) int32 { return t.Partitions[i].size() }) +} + +func (t topicMetadataV0) writeTo(w *bufio.Writer) { + writeInt16(w, t.TopicErrorCode) + writeString(w, t.TopicName) + writeArray(w, len(t.Partitions), func(i int) { t.Partitions[i].writeTo(w) }) +} + +type partitionMetadataV0 struct { + PartitionErrorCode int16 + PartitionID int32 + Leader int32 + Replicas []int32 + Isr []int32 +} + +func (p partitionMetadataV0) size() int32 { + return 2 + 4 + 4 + sizeofInt32Array(p.Replicas) + sizeofInt32Array(p.Isr) +} + +func (p partitionMetadataV0) writeTo(w *bufio.Writer) { + writeInt16(w, p.PartitionErrorCode) + writeInt32(w, p.PartitionID) + writeInt32(w, p.Leader) + writeInt32Array(w, p.Replicas) + writeInt32Array(w, p.Isr) +} diff --git a/vendor/github.com/segmentio/kafka-go/offsetcommit.go b/vendor/github.com/segmentio/kafka-go/offsetcommit.go new file mode 100644 index 000000000..4de7814ab --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/offsetcommit.go @@ -0,0 +1,177 @@ +package kafka + +import "bufio" + +type offsetCommitRequestV3Partition struct { + // Partition ID + Partition int32 + + // Offset to be committed + Offset int64 + + // Metadata holds any associated metadata the client wants to keep + Metadata string +} + +func (t offsetCommitRequestV3Partition) size() int32 { + return sizeofInt32(t.Partition) + + sizeofInt64(t.Offset) + + sizeofString(t.Metadata) +} + +func (t offsetCommitRequestV3Partition) writeTo(w *bufio.Writer) { + writeInt32(w, t.Partition) + writeInt64(w, t.Offset) + writeString(w, t.Metadata) +} + +type offsetCommitRequestV3Topic struct { + // Topic name + Topic string + + // Partitions to commit offsets + Partitions []offsetCommitRequestV3Partition +} + +func (t offsetCommitRequestV3Topic) size() int32 { + return sizeofString(t.Topic) + + sizeofArray(len(t.Partitions), func(i int) int32 { return t.Partitions[i].size() }) +} + +func (t offsetCommitRequestV3Topic) writeTo(w *bufio.Writer) { + writeString(w, t.Topic) + writeArray(w, len(t.Partitions), func(i int) { t.Partitions[i].writeTo(w) }) +} + +type offsetCommitRequestV3 struct { + // GroupID holds the unique group identifier + GroupID string + + // GenerationID holds the generation of the group. + GenerationID int32 + + // MemberID assigned by the group coordinator + MemberID string + + // RetentionTime holds the time period in ms to retain the offset. + RetentionTime int64 + + // Topics to commit offsets + Topics []offsetCommitRequestV3Topic +} + +func (t offsetCommitRequestV3) size() int32 { + return sizeofString(t.GroupID) + + sizeofInt32(t.GenerationID) + + sizeofString(t.MemberID) + + sizeofInt64(t.RetentionTime) + + sizeofArray(len(t.Topics), func(i int) int32 { return t.Topics[i].size() }) +} + +func (t offsetCommitRequestV3) writeTo(w *bufio.Writer) { + writeString(w, t.GroupID) + writeInt32(w, t.GenerationID) + writeString(w, t.MemberID) + writeInt64(w, t.RetentionTime) + writeArray(w, len(t.Topics), func(i int) { t.Topics[i].writeTo(w) }) +} + +type offsetCommitResponseV3PartitionResponse struct { + Partition int32 + + // ErrorCode holds response error code + ErrorCode int16 +} + +func (t offsetCommitResponseV3PartitionResponse) size() int32 { + return sizeofInt32(t.Partition) + + sizeofInt16(t.ErrorCode) +} + +func (t offsetCommitResponseV3PartitionResponse) writeTo(w *bufio.Writer) { + writeInt32(w, t.Partition) + writeInt16(w, t.ErrorCode) +} + +func (t *offsetCommitResponseV3PartitionResponse) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readInt32(r, size, &t.Partition); err != nil { + return + } + if remain, err = readInt16(r, remain, &t.ErrorCode); err != nil { + return + } + return +} + +type offsetCommitResponseV3Response struct { + Topic string + PartitionResponses []offsetCommitResponseV3PartitionResponse +} + +func (t offsetCommitResponseV3Response) size() int32 { + return sizeofString(t.Topic) + + sizeofArray(len(t.PartitionResponses), func(i int) int32 { return t.PartitionResponses[i].size() }) +} + +func (t offsetCommitResponseV3Response) writeTo(w *bufio.Writer) { + writeString(w, t.Topic) + writeArray(w, len(t.PartitionResponses), func(i int) { t.PartitionResponses[i].writeTo(w) }) +} + +func (t *offsetCommitResponseV3Response) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readString(r, size, &t.Topic); err != nil { + return + } + + fn := func(r *bufio.Reader, withSize int) (fnRemain int, fnErr error) { + item := offsetCommitResponseV3PartitionResponse{} + if fnRemain, fnErr = (&item).readFrom(r, withSize); fnErr != nil { + return + } + t.PartitionResponses = append(t.PartitionResponses, item) + return + } + if remain, err = readArrayWith(r, remain, fn); err != nil { + return + } + + return +} + +type offsetCommitResponseV3 struct { + // ThrottleTimeMS holds the duration in milliseconds for which the request + // was throttled due to quota violation (Zero if the request did not violate + // any quota) + ThrottleTimeMS int32 + Responses []offsetCommitResponseV3Response +} + +func (t offsetCommitResponseV3) size() int32 { + return sizeofInt32(t.ThrottleTimeMS) + + sizeofArray(len(t.Responses), func(i int) int32 { return t.Responses[i].size() }) +} + +func (t offsetCommitResponseV3) writeTo(w *bufio.Writer) { + writeInt32(w, t.ThrottleTimeMS) + writeArray(w, len(t.Responses), func(i int) { t.Responses[i].writeTo(w) }) +} + +func (t *offsetCommitResponseV3) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readInt32(r, size, &t.ThrottleTimeMS); err != nil { + return + } + + fn := func(r *bufio.Reader, withSize int) (fnRemain int, fnErr error) { + item := offsetCommitResponseV3Response{} + if fnRemain, fnErr = (&item).readFrom(r, withSize); fnErr != nil { + return + } + t.Responses = append(t.Responses, item) + return + } + if remain, err = readArrayWith(r, remain, fn); err != nil { + return + } + + return +} diff --git a/vendor/github.com/segmentio/kafka-go/offsetfetch.go b/vendor/github.com/segmentio/kafka-go/offsetfetch.go new file mode 100644 index 000000000..7931490da --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/offsetfetch.go @@ -0,0 +1,188 @@ +package kafka + +import ( + "bufio" +) + +type offsetFetchRequestV3Topic struct { + // Topic name + Topic string + + // Partitions to fetch offsets + Partitions []int32 +} + +func (t offsetFetchRequestV3Topic) size() int32 { + return sizeofString(t.Topic) + + sizeofInt32Array(t.Partitions) +} + +func (t offsetFetchRequestV3Topic) writeTo(w *bufio.Writer) { + writeString(w, t.Topic) + writeInt32Array(w, t.Partitions) +} + +type offsetFetchRequestV3 struct { + // GroupID holds the unique group identifier + GroupID string + + // Topics to fetch offsets. + Topics []offsetFetchRequestV3Topic +} + +func (t offsetFetchRequestV3) size() int32 { + return sizeofString(t.GroupID) + + sizeofArray(len(t.Topics), func(i int) int32 { return t.Topics[i].size() }) +} + +func (t offsetFetchRequestV3) writeTo(w *bufio.Writer) { + writeString(w, t.GroupID) + writeArray(w, len(t.Topics), func(i int) { t.Topics[i].writeTo(w) }) +} + +type offsetFetchResponseV3PartitionResponse struct { + // Partition ID + Partition int32 + + // Offset of last committed message + Offset int64 + + // Metadata client wants to keep + Metadata string + + // ErrorCode holds response error code + ErrorCode int16 +} + +func (t offsetFetchResponseV3PartitionResponse) size() int32 { + return sizeofInt32(t.Partition) + + sizeofInt64(t.Offset) + + sizeofString(t.Metadata) + + sizeofInt16(t.ErrorCode) +} + +func (t offsetFetchResponseV3PartitionResponse) writeTo(w *bufio.Writer) { + writeInt32(w, t.Partition) + writeInt64(w, t.Offset) + writeString(w, t.Metadata) + writeInt16(w, t.ErrorCode) +} + +func (t *offsetFetchResponseV3PartitionResponse) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readInt32(r, size, &t.Partition); err != nil { + return + } + if remain, err = readInt64(r, remain, &t.Offset); err != nil { + return + } + if remain, err = readString(r, remain, &t.Metadata); err != nil { + return + } + if remain, err = readInt16(r, remain, &t.ErrorCode); err != nil { + return + } + return +} + +type offsetFetchResponseV3Response struct { + // Topic name + Topic string + + // PartitionResponses holds offsets by partition + PartitionResponses []offsetFetchResponseV3PartitionResponse +} + +func (t offsetFetchResponseV3Response) size() int32 { + return sizeofString(t.Topic) + + sizeofArray(len(t.PartitionResponses), func(i int) int32 { return t.PartitionResponses[i].size() }) +} + +func (t offsetFetchResponseV3Response) writeTo(w *bufio.Writer) { + writeString(w, t.Topic) + writeArray(w, len(t.PartitionResponses), func(i int) { t.PartitionResponses[i].writeTo(w) }) +} + +func (t *offsetFetchResponseV3Response) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readString(r, size, &t.Topic); err != nil { + return + } + + fn := func(r *bufio.Reader, size int) (fnRemain int, fnErr error) { + item := offsetFetchResponseV3PartitionResponse{} + if fnRemain, fnErr = (&item).readFrom(r, size); err != nil { + return + } + t.PartitionResponses = append(t.PartitionResponses, item) + return + } + if remain, err = readArrayWith(r, remain, fn); err != nil { + return + } + + return +} + +type offsetFetchResponseV3 struct { + // ThrottleTimeMS holds the duration in milliseconds for which the request + // was throttled due to quota violation (Zero if the request did not violate + // any quota) + ThrottleTimeMS int32 + + // Responses holds topic partition offsets + Responses []offsetFetchResponseV3Response + + // ErrorCode holds response error code + ErrorCode int16 +} + +func (t offsetFetchResponseV3) size() int32 { + return sizeofInt32(t.ThrottleTimeMS) + + sizeofArray(len(t.Responses), func(i int) int32 { return t.Responses[i].size() }) + + sizeofInt16(t.ErrorCode) +} + +func (t offsetFetchResponseV3) writeTo(w *bufio.Writer) { + writeInt32(w, t.ThrottleTimeMS) + writeArray(w, len(t.Responses), func(i int) { t.Responses[i].writeTo(w) }) + writeInt16(w, t.ErrorCode) +} + +func (t *offsetFetchResponseV3) readFrom(r *bufio.Reader, size int) (remain int, err error) { + if remain, err = readInt32(r, size, &t.ThrottleTimeMS); err != nil { + return + } + + fn := func(r *bufio.Reader, withSize int) (fnRemain int, fnErr error) { + item := offsetFetchResponseV3Response{} + if fnRemain, fnErr = (&item).readFrom(r, withSize); fnErr != nil { + return + } + t.Responses = append(t.Responses, item) + return + } + if remain, err = readArrayWith(r, remain, fn); err != nil { + return + } + + if remain, err = readInt16(r, remain, &t.ErrorCode); err != nil { + return + } + + return +} + +func findOffset(topic string, partition int32, response offsetFetchResponseV3) (int64, bool) { + for _, r := range response.Responses { + if r.Topic != topic { + continue + } + + for _, pr := range r.PartitionResponses { + if pr.Partition == partition { + return pr.Offset, true + } + } + } + + return 0, false +} diff --git a/vendor/github.com/segmentio/kafka-go/produce.go b/vendor/github.com/segmentio/kafka-go/produce.go new file mode 100644 index 000000000..0eda7c811 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/produce.go @@ -0,0 +1,113 @@ +package kafka + +import "bufio" + +type produceRequestV2 struct { + RequiredAcks int16 + Timeout int32 + Topics []produceRequestTopicV2 +} + +func (r produceRequestV2) size() int32 { + return 2 + 4 + sizeofArray(len(r.Topics), func(i int) int32 { return r.Topics[i].size() }) +} + +func (r produceRequestV2) writeTo(w *bufio.Writer) { + writeInt16(w, r.RequiredAcks) + writeInt32(w, r.Timeout) + writeArray(w, len(r.Topics), func(i int) { r.Topics[i].writeTo(w) }) +} + +type produceRequestTopicV2 struct { + TopicName string + Partitions []produceRequestPartitionV2 +} + +func (t produceRequestTopicV2) size() int32 { + return sizeofString(t.TopicName) + + sizeofArray(len(t.Partitions), func(i int) int32 { return t.Partitions[i].size() }) +} + +func (t produceRequestTopicV2) writeTo(w *bufio.Writer) { + writeString(w, t.TopicName) + writeArray(w, len(t.Partitions), func(i int) { t.Partitions[i].writeTo(w) }) +} + +type produceRequestPartitionV2 struct { + Partition int32 + MessageSetSize int32 + MessageSet messageSet +} + +func (p produceRequestPartitionV2) size() int32 { + return 4 + 4 + p.MessageSet.size() +} + +func (p produceRequestPartitionV2) writeTo(w *bufio.Writer) { + writeInt32(w, p.Partition) + writeInt32(w, p.MessageSetSize) + p.MessageSet.writeTo(w) +} + +type produceResponseV2 struct { + ThrottleTime int32 + Topics []produceResponseTopicV2 +} + +func (r produceResponseV2) size() int32 { + return 4 + sizeofArray(len(r.Topics), func(i int) int32 { return r.Topics[i].size() }) +} + +func (r produceResponseV2) writeTo(w *bufio.Writer) { + writeInt32(w, r.ThrottleTime) + writeArray(w, len(r.Topics), func(i int) { r.Topics[i].writeTo(w) }) +} + +type produceResponseTopicV2 struct { + TopicName string + Partitions []produceResponsePartitionV2 +} + +func (t produceResponseTopicV2) size() int32 { + return sizeofString(t.TopicName) + + sizeofArray(len(t.Partitions), func(i int) int32 { return t.Partitions[i].size() }) +} + +func (t produceResponseTopicV2) writeTo(w *bufio.Writer) { + writeString(w, t.TopicName) + writeArray(w, len(t.Partitions), func(i int) { t.Partitions[i].writeTo(w) }) +} + +type produceResponsePartitionV2 struct { + Partition int32 + ErrorCode int16 + Offset int64 + Timestamp int64 +} + +func (p produceResponsePartitionV2) size() int32 { + return 4 + 2 + 8 + 8 +} + +func (p produceResponsePartitionV2) writeTo(w *bufio.Writer) { + writeInt32(w, p.Partition) + writeInt16(w, p.ErrorCode) + writeInt64(w, p.Offset) + writeInt64(w, p.Timestamp) +} + +func (p *produceResponsePartitionV2) readFrom(r *bufio.Reader, sz int) (remain int, err error) { + if remain, err = readInt32(r, sz, &p.Partition); err != nil { + return + } + if remain, err = readInt16(r, remain, &p.ErrorCode); err != nil { + return + } + if remain, err = readInt64(r, remain, &p.Offset); err != nil { + return + } + if remain, err = readInt64(r, remain, &p.Timestamp); err != nil { + return + } + return +} diff --git a/vendor/github.com/segmentio/kafka-go/protocol.go b/vendor/github.com/segmentio/kafka-go/protocol.go new file mode 100644 index 000000000..30d987aff --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/protocol.go @@ -0,0 +1,84 @@ +package kafka + +import ( + "bufio" + "encoding/binary" + "fmt" +) + +type apiKey int16 + +const ( + produceRequest apiKey = 0 + fetchRequest apiKey = 1 + listOffsetRequest apiKey = 2 + metadataRequest apiKey = 3 + offsetCommitRequest apiKey = 8 + offsetFetchRequest apiKey = 9 + groupCoordinatorRequest apiKey = 10 + joinGroupRequest apiKey = 11 + heartbeatRequest apiKey = 12 + leaveGroupRequest apiKey = 13 + syncGroupRequest apiKey = 14 + describeGroupsRequest apiKey = 15 + listGroupsRequest apiKey = 16 + createTopicsRequest apiKey = 19 + deleteTopicsRequest apiKey = 20 +) + +type apiVersion int16 + +const ( + v0 apiVersion = 0 + v1 apiVersion = 1 + v2 apiVersion = 2 + v3 apiVersion = 3 +) + +type requestHeader struct { + Size int32 + ApiKey int16 + ApiVersion int16 + CorrelationID int32 + ClientID string +} + +func (h requestHeader) size() int32 { + return 4 + 2 + 2 + 4 + sizeofString(h.ClientID) +} + +func (h requestHeader) writeTo(w *bufio.Writer) { + writeInt32(w, h.Size) + writeInt16(w, h.ApiKey) + writeInt16(w, h.ApiVersion) + writeInt32(w, h.CorrelationID) + writeString(w, h.ClientID) +} + +type request interface { + size() int32 + writeTo(*bufio.Writer) +} + +func makeInt8(b []byte) int8 { + return int8(b[0]) +} + +func makeInt16(b []byte) int16 { + return int16(binary.BigEndian.Uint16(b)) +} + +func makeInt32(b []byte) int32 { + return int32(binary.BigEndian.Uint32(b)) +} + +func makeInt64(b []byte) int64 { + return int64(binary.BigEndian.Uint64(b)) +} + +func expectZeroSize(sz int, err error) error { + if err == nil && sz != 0 { + err = fmt.Errorf("reading a response left %d unread bytes", sz) + } + return err +} diff --git a/vendor/github.com/segmentio/kafka-go/read.go b/vendor/github.com/segmentio/kafka-go/read.go new file mode 100644 index 000000000..d0b323819 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/read.go @@ -0,0 +1,400 @@ +package kafka + +import ( + "bufio" + "errors" + "fmt" + "io" + "reflect" +) + +type readable interface { + readFrom(*bufio.Reader, int) (int, error) +} + +var errShortRead = errors.New("not enough bytes available to load the response") + +func peekRead(r *bufio.Reader, sz int, n int, f func([]byte)) (int, error) { + if n > sz { + return sz, errShortRead + } + b, err := r.Peek(n) + if err != nil { + return sz, err + } + f(b) + return discardN(r, sz, n) +} + +func readInt8(r *bufio.Reader, sz int, v *int8) (int, error) { + return peekRead(r, sz, 1, func(b []byte) { *v = makeInt8(b) }) +} + +func readInt16(r *bufio.Reader, sz int, v *int16) (int, error) { + return peekRead(r, sz, 2, func(b []byte) { *v = makeInt16(b) }) +} + +func readInt32(r *bufio.Reader, sz int, v *int32) (int, error) { + return peekRead(r, sz, 4, func(b []byte) { *v = makeInt32(b) }) +} + +func readInt64(r *bufio.Reader, sz int, v *int64) (int, error) { + return peekRead(r, sz, 8, func(b []byte) { *v = makeInt64(b) }) +} + +func readString(r *bufio.Reader, sz int, v *string) (int, error) { + return readStringWith(r, sz, func(r *bufio.Reader, sz int, n int) (remain int, err error) { + *v, remain, err = readNewString(r, sz, n) + return + }) +} + +func readStringWith(r *bufio.Reader, sz int, cb func(*bufio.Reader, int, int) (int, error)) (int, error) { + var err error + var len int16 + + if sz, err = readInt16(r, sz, &len); err != nil { + return sz, err + } + + n := int(len) + if n > sz { + return sz, errShortRead + } + + return cb(r, sz, n) +} + +func readNewString(r *bufio.Reader, sz int, n int) (string, int, error) { + b, sz, err := readNewBytes(r, sz, n) + return string(b), sz, err +} + +func readBytes(r *bufio.Reader, sz int, v *[]byte) (int, error) { + return readBytesWith(r, sz, func(r *bufio.Reader, sz int, n int) (remain int, err error) { + *v, remain, err = readNewBytes(r, sz, n) + return + }) +} + +func readBytesWith(r *bufio.Reader, sz int, cb func(*bufio.Reader, int, int) (int, error)) (int, error) { + var err error + var len int32 + + if sz, err = readInt32(r, sz, &len); err != nil { + return sz, err + } + + n := int(len) + if n > sz { + return sz, errShortRead + } + + return cb(r, sz, n) +} + +func readNewBytes(r *bufio.Reader, sz int, n int) ([]byte, int, error) { + var err error + var b []byte + + if n > 0 { + b = make([]byte, n) + n, err = io.ReadFull(r, b) + b = b[:n] + sz -= n + } + + return b, sz, err +} + +func readArrayWith(r *bufio.Reader, sz int, cb func(*bufio.Reader, int) (int, error)) (int, error) { + var err error + var len int32 + + if sz, err = readInt32(r, sz, &len); err != nil { + return sz, err + } + + for n := int(len); n > 0; n-- { + if sz, err = cb(r, sz); err != nil { + break + } + } + + return sz, err +} + +func readStringArray(r *bufio.Reader, sz int, v *[]string) (remain int, err error) { + var content []string + fn := func(r *bufio.Reader, size int) (fnRemain int, fnErr error) { + var value string + if fnRemain, fnErr = readString(r, size, &value); fnErr != nil { + return + } + content = append(content, value) + return + } + if remain, err = readArrayWith(r, sz, fn); err != nil { + return + } + + *v = content + return +} + +func readMapStringInt32(r *bufio.Reader, sz int, v *map[string][]int32) (remain int, err error) { + var len int32 + if remain, err = readInt32(r, sz, &len); err != nil { + return + } + + content := make(map[string][]int32, len) + for i := 0; i < int(len); i++ { + var key string + var values []int32 + + if remain, err = readString(r, remain, &key); err != nil { + return + } + + fn := func(r *bufio.Reader, size int) (fnRemain int, fnErr error) { + var value int32 + if fnRemain, fnErr = readInt32(r, size, &value); fnErr != nil { + return + } + values = append(values, value) + return + } + if remain, err = readArrayWith(r, remain, fn); err != nil { + return + } + + content[key] = values + } + *v = content + + return +} + +func read(r *bufio.Reader, sz int, a interface{}) (int, error) { + switch v := a.(type) { + case *int8: + return readInt8(r, sz, v) + case *int16: + return readInt16(r, sz, v) + case *int32: + return readInt32(r, sz, v) + case *int64: + return readInt64(r, sz, v) + case *string: + return readString(r, sz, v) + case *[]byte: + return readBytes(r, sz, v) + } + switch v := reflect.ValueOf(a).Elem(); v.Kind() { + case reflect.Struct: + return readStruct(r, sz, v) + case reflect.Slice: + return readSlice(r, sz, v) + default: + panic(fmt.Sprintf("unsupported type: %T", a)) + } +} + +func readAll(r *bufio.Reader, sz int, ptrs ...interface{}) (int, error) { + var err error + + for _, ptr := range ptrs { + if sz, err = readPtr(r, sz, ptr); err != nil { + break + } + } + + return sz, err +} + +func readPtr(r *bufio.Reader, sz int, ptr interface{}) (int, error) { + switch v := ptr.(type) { + case *int8: + return readInt8(r, sz, v) + case *int16: + return readInt16(r, sz, v) + case *int32: + return readInt32(r, sz, v) + case *int64: + return readInt64(r, sz, v) + case *string: + return readString(r, sz, v) + case *[]byte: + return readBytes(r, sz, v) + case readable: + return v.readFrom(r, sz) + default: + panic(fmt.Sprintf("unsupported type: %T", v)) + } +} + +func readStruct(r *bufio.Reader, sz int, v reflect.Value) (int, error) { + var err error + for i, n := 0, v.NumField(); i != n; i++ { + if sz, err = read(r, sz, v.Field(i).Addr().Interface()); err != nil { + return sz, err + } + } + return sz, nil +} + +func readSlice(r *bufio.Reader, sz int, v reflect.Value) (int, error) { + var err error + var len int32 + + if sz, err = readInt32(r, sz, &len); err != nil { + return sz, err + } + + if n := int(len); n < 0 { + v.Set(reflect.Zero(v.Type())) + } else { + v.Set(reflect.MakeSlice(v.Type(), n, n)) + + for i := 0; i != n; i++ { + if sz, err = read(r, sz, v.Index(i).Addr().Interface()); err != nil { + return sz, err + } + } + } + + return sz, nil +} + +func readFetchResponseHeader(r *bufio.Reader, size int) (throttle int32, watermark int64, remain int, err error) { + var n int32 + var p struct { + Partition int32 + ErrorCode int16 + HighwaterMarkOffset int64 + MessageSetSize int32 + } + + if remain, err = readInt32(r, size, &throttle); err != nil { + return + } + + if remain, err = readInt32(r, remain, &n); err != nil { + return + } + + // This error should never trigger, unless there's a bug in the kafka client + // or server. + if n != 1 { + err = fmt.Errorf("1 kafka topic was expected in the fetch response but the client received %d", n) + return + } + + // We ignore the topic name because we've requests messages for a single + // topic, unless there's a bug in the kafka server we will have received + // the name of the topic that we requested. + if remain, err = discardString(r, remain); err != nil { + return + } + + if remain, err = readInt32(r, remain, &n); err != nil { + return + } + + // This error should never trigger, unless there's a bug in the kafka client + // or server. + if n != 1 { + err = fmt.Errorf("1 kafka partition was expected in the fetch response but the client received %d", n) + return + } + + if remain, err = read(r, remain, &p); err != nil { + return + } + + if p.ErrorCode != 0 { + err = Error(p.ErrorCode) + return + } + + // This error should never trigger, unless there's a bug in the kafka client + // or server. + if remain != int(p.MessageSetSize) { + err = fmt.Errorf("the size of the message set in a fetch response doesn't match the number of remaining bytes (message set size = %d, remaining bytes = %d)", p.MessageSetSize, remain) + return + } + + watermark = p.HighwaterMarkOffset + return +} + +func readMessageHeader(r *bufio.Reader, sz int) (offset int64, attributes int8, timestamp int64, remain int, err error) { + var version int8 + + if remain, err = readInt64(r, sz, &offset); err != nil { + return + } + + // On discarding the message size and CRC: + // --------------------------------------- + // + // - Not sure why kafka gives the message size here, we already have the + // number of remaining bytes in the response and kafka should only truncate + // the trailing message. + // + // - TCP is already taking care of ensuring data integrity, no need to + // waste resources doing it a second time so we just skip the message CRC. + // + if remain, err = discardN(r, remain, 8); err != nil { + return + } + + if remain, err = readInt8(r, remain, &version); err != nil { + return + } + + if remain, err = readInt8(r, remain, &attributes); err != nil { + return + } + + switch version { + case 0: + case 1: + remain, err = readInt64(r, remain, ×tamp) + default: + err = fmt.Errorf("unsupported message version %d found in fetch response", version) + } + + return +} + +func readMessage(r *bufio.Reader, sz int, min int64, + key func(*bufio.Reader, int, int) (int, error), + val func(*bufio.Reader, int, int) (int, error), +) (offset int64, timestamp int64, remain int, err error) { + for { + // TODO: read attributes and decompress the message + if offset, _, timestamp, remain, err = readMessageHeader(r, sz); err != nil { + return + } + + // When the messages are compressed kafka may return messages at an + // earlier offset than the one that was requested, apparently it's the + // client's responsibility to ignore those. + if offset >= min { + if remain, err = readBytesWith(r, remain, key); err != nil { + return + } + remain, err = readBytesWith(r, remain, val) + return + } + + if remain, err = discardBytes(r, remain); err != nil { + return + } + if remain, err = discardBytes(r, remain); err != nil { + return + } + } +} diff --git a/vendor/github.com/segmentio/kafka-go/reader.go b/vendor/github.com/segmentio/kafka-go/reader.go new file mode 100644 index 000000000..20d79b344 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/reader.go @@ -0,0 +1,1842 @@ +package kafka + +import ( + "bufio" + "bytes" + "context" + "errors" + "fmt" + "io" + "log" + "math" + "sort" + "strconv" + "sync" + "sync/atomic" + "time" +) + +const ( + firstOffset = -1 + lastOffset = -2 +) + +const ( + // defaultCommitRetries holds the number commit attempts to make + // before giving up + defaultCommitRetries = 3 +) + +var ( + errOnlyAvailableWithGroup = errors.New("unavailable when GroupID is not set") + errNotAvailableWithGroup = errors.New("unavailable when GroupID is set") +) + +const ( + // defaultProtocolType holds the default protocol type documented in the + // kafka protocol + // + // See https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-GroupMembershipAPI + defaultProtocolType = "consumer" + + // defaultHeartbeatInterval contains the default time between heartbeats. If + // the coordinator does not receive a heartbeat within the session timeout interval, + // the consumer will be considered dead and the coordinator will rebalance the + // group. + // + // As a rule, the heartbeat interval should be no greater than 1/3 the session timeout + defaultHeartbeatInterval = 3 * time.Second + + // defaultSessionTimeout contains the default interval the coordinator will wait + // for a heartbeat before marking a consumer as dead + defaultSessionTimeout = 30 * time.Second + + // defaultRebalanceTimeout contains the amount of time the coordinator will wait + // for consumers to issue a join group once a rebalance has been requested + defaultRebalanceTimeout = 30 * time.Second + + // defaultRetentionTime holds the length of time a the consumer group will be + // saved by kafka + defaultRetentionTime = time.Hour * 24 +) + +// Reader provides a high-level API for consuming messages from kafka. +// +// A Reader automatically manages reconnections to a kafka server, and +// blocking methods have context support for asynchronous cancellations. +type Reader struct { + // immutable fields of the reader + config ReaderConfig + + // communication channels between the parent reader and its subreaders + msgs chan readerMessage + + // mutable fields of the reader (synchronized on the mutex) + mutex sync.Mutex + join sync.WaitGroup + cancel context.CancelFunc + stop context.CancelFunc + done chan struct{} + commits chan commitRequest + version int64 // version holds the generation of the spawned readers + offset int64 + lag int64 + closed bool + address string // address of group coordinator + generationID int32 // generationID of group + memberID string // memberID of group + + // offsetStash should only be managed by the commitLoopInterval. We store + // it here so that it survives rebalances + offsetStash offsetStash + + // reader stats are all made of atomic values, no need for synchronization. + once uint32 + stctx context.Context + // reader stats are all made of atomic values, no need for synchronization. + // Use a pointer to ensure 64-bit alignment of the values. + stats *readerStats +} + +// useConsumerGroup indicates whether the Reader is part of a consumer group. +func (r *Reader) useConsumerGroup() bool { return r.config.GroupID != "" } + +// useSyncCommits indicates whether the Reader is configured to perform sync or +// async commits. +func (r *Reader) useSyncCommits() bool { return r.config.CommitInterval == 0 } + +// membership returns the group generationID and memberID of the reader. +// +// Only used when config.GroupID != "" +func (r *Reader) membership() (generationID int32, memberID string) { + r.mutex.Lock() + generationID = r.generationID + memberID = r.memberID + r.mutex.Unlock() + return +} + +// lookupCoordinator scans the brokers and looks up the address of the +// coordinator for the group. +// +// Only used when config.GroupID != "" +func (r *Reader) lookupCoordinator() (string, error) { + conn, err := r.connect() + if err != nil { + return "", fmt.Errorf("unable to coordinator to any connect for group, %v: %v\n", r.config.GroupID, err) + } + defer conn.Close() + + out, err := conn.findCoordinator(findCoordinatorRequestV1{ + CoordinatorKey: r.config.GroupID, + }) + if err != nil { + return "", fmt.Errorf("unable to find coordinator for group, %v: %v", r.config.GroupID, err) + } + + address := fmt.Sprintf("%v:%v", out.Coordinator.Host, out.Coordinator.Port) + return address, nil +} + +// refreshCoordinator updates the value of r.address +func (r *Reader) refreshCoordinator() (err error) { + const ( + backoffDelayMin = 100 * time.Millisecond + backoffDelayMax = 1 * time.Second + ) + + for attempt := 0; true; attempt++ { + if attempt != 0 { + if !sleep(r.stctx, backoff(attempt, backoffDelayMin, backoffDelayMax)) { + return r.stctx.Err() + } + } + + address, err := r.lookupCoordinator() + if err != nil { + continue + } + + r.mutex.Lock() + oldAddress := r.address + r.address = address + r.mutex.Unlock() + + if address != oldAddress { + r.withLogger(func(l *log.Logger) { + l.Printf("coordinator for group, %v, set to %v\n", r.config.GroupID, address) + }) + } + + break + } + + return nil +} + +// makeJoinGroupRequestV2 handles the logic of constructing a joinGroup +// request +func (r *Reader) makeJoinGroupRequestV2() (joinGroupRequestV2, error) { + _, memberID := r.membership() + + request := joinGroupRequestV2{ + GroupID: r.config.GroupID, + MemberID: memberID, + SessionTimeout: int32(r.config.SessionTimeout / time.Millisecond), + RebalanceTimeout: int32(r.config.RebalanceTimeout / time.Millisecond), + ProtocolType: defaultProtocolType, + } + + for _, strategy := range allStrategies { + meta, err := strategy.GroupMetadata([]string{r.config.Topic}) + if err != nil { + return joinGroupRequestV2{}, fmt.Errorf("unable to construct protocol metadata for member, %v: %v\n", strategy.ProtocolName(), err) + } + + request.GroupProtocols = append(request.GroupProtocols, joinGroupRequestGroupProtocolV2{ + ProtocolName: strategy.ProtocolName(), + ProtocolMetadata: meta.bytes(), + }) + } + + return request, nil +} + +// makeMemberProtocolMetadata maps encoded member metadata ([]byte) into memberGroupMetadata +func (r *Reader) makeMemberProtocolMetadata(in []joinGroupResponseMemberV2) ([]memberGroupMetadata, error) { + members := make([]memberGroupMetadata, 0, len(in)) + for _, item := range in { + metadata := groupMetadata{} + reader := bufio.NewReader(bytes.NewReader(item.MemberMetadata)) + if remain, err := (&metadata).readFrom(reader, len(item.MemberMetadata)); err != nil || remain != 0 { + return nil, fmt.Errorf("unable to read metadata for member, %v: %v\n", item.MemberID, err) + } + + member := memberGroupMetadata{ + MemberID: item.MemberID, + Metadata: metadata, + } + members = append(members, member) + } + return members, nil +} + +// partitionReader is an internal interface used to simplify unit testing +type partitionReader interface { + // ReadPartitions mirrors Conn.ReadPartitions + ReadPartitions(topics ...string) (partitions []Partition, err error) +} + +// assignTopicPartitions uses the selected strategy to assign members to their +// various partitions +func (r *Reader) assignTopicPartitions(conn partitionReader, group joinGroupResponseV2) (memberGroupAssignments, error) { + r.withLogger(func(l *log.Logger) { + l.Println("selected as leader for group,", r.config.GroupID) + }) + + strategy, ok := findStrategy(group.GroupProtocol, allStrategies) + if !ok { + return nil, fmt.Errorf("unable to find selected strategy, %v, for group, %v", group.GroupProtocol, r.config.GroupID) + } + + members, err := r.makeMemberProtocolMetadata(group.Members) + if err != nil { + return nil, fmt.Errorf("unable to construct MemberProtocolMetadata: %v", err) + } + + topics := extractTopics(members) + partitions, err := conn.ReadPartitions(topics...) + if err != nil { + return nil, fmt.Errorf("unable to read partitions: %v", err) + } + + r.withLogger(func(l *log.Logger) { + l.Printf("using '%v' strategy to assign group, %v\n", group.GroupProtocol, r.config.GroupID) + for _, partition := range partitions { + l.Printf("found topic/partition: %v/%v", partition.Topic, partition.ID) + } + }) + + return strategy.AssignGroups(members, partitions), nil +} + +func (r *Reader) leaveGroup(conn *Conn) error { + _, memberID := r.membership() + _, err := conn.leaveGroup(leaveGroupRequestV1{ + GroupID: r.config.GroupID, + MemberID: memberID, + }) + if err != nil { + return fmt.Errorf("leave group failed for group, %v, and member, %v: %v", r.config.GroupID, memberID, err) + } + + return nil +} + +// joinGroup attempts to join the reader to the consumer group. +// Returns memberGroupAssignments is this Reader was selected as +// the leader. Otherwise, memberGroupAssignments will be nil. +// +// Possible kafka error codes returned: +// * GroupLoadInProgress: +// * GroupCoordinatorNotAvailable: +// * NotCoordinatorForGroup: +// * InconsistentGroupProtocol: +// * InvalidSessionTimeout: +// * GroupAuthorizationFailed: +func (r *Reader) joinGroup() (memberGroupAssignments, error) { + conn, err := r.coordinator() + if err != nil { + return nil, err + } + defer conn.Close() + + request, err := r.makeJoinGroupRequestV2() + if err != nil { + return nil, err + } + + response, err := conn.joinGroup(request) + if err != nil { + switch err { + case UnknownMemberId: + r.mutex.Lock() + r.memberID = "" + r.mutex.Unlock() + return nil, fmt.Errorf("joinGroup failed: %v", err) + + default: + return nil, fmt.Errorf("joinGroup failed: %v", err) + } + } + + // Extract our membership and generationID from the response + r.mutex.Lock() + oldGenerationID := r.generationID + oldMemberID := r.memberID + r.generationID = response.GenerationID + r.memberID = response.MemberID + r.mutex.Unlock() + + if oldGenerationID != response.GenerationID || oldMemberID != response.MemberID { + r.withLogger(func(l *log.Logger) { + l.Printf("response membership changed. generationID: %v => %v, memberID: '%v' => '%v'\n", + oldGenerationID, + response.GenerationID, + oldMemberID, + response.MemberID, + ) + }) + } + + var assignments memberGroupAssignments + if iAmLeader := response.MemberID == response.LeaderID; iAmLeader { + v, err := r.assignTopicPartitions(conn, response) + if err != nil { + _ = r.leaveGroup(conn) + return nil, err + } + assignments = v + + r.withLogger(func(l *log.Logger) { + for memberID, assignment := range assignments { + for topic, partitions := range assignment { + l.Printf("assigned member/topic/partitions %v/%v/%v\n", memberID, topic, partitions) + } + } + }) + } + + r.withLogger(func(l *log.Logger) { + l.Printf("joinGroup succeeded for response, %v. generationID=%v, memberID=%v\n", r.config.GroupID, response.GenerationID, response.MemberID) + }) + + return assignments, nil +} + +func (r *Reader) makeSyncGroupRequestV1(memberAssignments memberGroupAssignments) syncGroupRequestV1 { + generationID, memberID := r.membership() + request := syncGroupRequestV1{ + GroupID: r.config.GroupID, + GenerationID: generationID, + MemberID: memberID, + } + + if memberAssignments != nil { + request.GroupAssignments = make([]syncGroupRequestGroupAssignmentV1, 0, 1) + + for memberID, topics := range memberAssignments { + request.GroupAssignments = append(request.GroupAssignments, syncGroupRequestGroupAssignmentV1{ + MemberID: memberID, + MemberAssignments: groupAssignment{ + Version: 1, + Topics: topics, + }.bytes(), + }) + } + } + + return request +} + +// syncGroup completes the consumer group handshake by accepting the +// memberAssignments (if this Reader is the leader) and returning this +// Readers subscriptions topic => partitions +// +// Possible kafka error codes returned: +// * GroupCoordinatorNotAvailable: +// * NotCoordinatorForGroup: +// * IllegalGeneration: +// * RebalanceInProgress: +// * GroupAuthorizationFailed: +func (r *Reader) syncGroup(memberAssignments memberGroupAssignments) (map[string][]int32, error) { + conn, err := r.coordinator() + if err != nil { + return nil, err + } + defer conn.Close() + + request := r.makeSyncGroupRequestV1(memberAssignments) + response, err := conn.syncGroups(request) + if err != nil { + switch err { + case RebalanceInProgress: + // don't leave the group + return nil, fmt.Errorf("syncGroup failed: %v", err) + + case UnknownMemberId: + r.mutex.Lock() + r.memberID = "" + r.mutex.Unlock() + _ = r.leaveGroup(conn) + return nil, fmt.Errorf("syncGroup failed: %v", err) + + default: + _ = r.leaveGroup(conn) + return nil, fmt.Errorf("syncGroup failed: %v", err) + } + } + + assignments := groupAssignment{} + reader := bufio.NewReader(bytes.NewReader(response.MemberAssignments)) + if _, err := (&assignments).readFrom(reader, len(response.MemberAssignments)); err != nil { + _ = r.leaveGroup(conn) + return nil, fmt.Errorf("unable to read SyncGroup response for group, %v: %v\n", r.config.GroupID, err) + } + + r.withLogger(func(l *log.Logger) { + l.Printf("sync group finished for group, %v\n", r.config.GroupID) + }) + + return assignments.Topics, nil +} + +func (r *Reader) rebalance() (map[string][]int32, error) { + r.withLogger(func(l *log.Logger) { + l.Printf("rebalancing consumer group, %v", r.config.GroupID) + }) + + if err := r.refreshCoordinator(); err != nil { + return nil, err + } + + members, err := r.joinGroup() + if err != nil { + return nil, err + } + + assignments, err := r.syncGroup(members) + if err != nil { + return nil, err + } + + return assignments, nil +} + +func (r *Reader) unsubscribe() error { + r.cancel() + r.join.Wait() + return nil +} + +func (r *Reader) fetchOffsets(subs map[string][]int32) (map[int]int64, error) { + conn, err := r.coordinator() + if err != nil { + return nil, err + } + defer conn.Close() + + partitions := subs[r.config.Topic] + offsets, err := conn.offsetFetch(offsetFetchRequestV3{ + GroupID: r.config.GroupID, + Topics: []offsetFetchRequestV3Topic{ + { + Topic: r.config.Topic, + Partitions: partitions, + }, + }, + }) + if err != nil { + return nil, err + } + + offsetsByPartition := map[int]int64{} + for _, pr := range offsets.Responses[0].PartitionResponses { + for _, partition := range partitions { + if partition == pr.Partition { + offset := pr.Offset + offsetsByPartition[int(partition)] = offset + } + } + } + + return offsetsByPartition, nil +} + +func (r *Reader) subscribe(subs map[string][]int32) error { + if len(subs[r.config.Topic]) == 0 { + return nil + } + + offsetsByPartition, err := r.fetchOffsets(subs) + if err != nil { + if conn, err := r.coordinator(); err == nil { + // make an attempt at leaving the group + _ = r.leaveGroup(conn) + conn.Close() + } + + return err + } + + r.mutex.Lock() + r.start(offsetsByPartition) + r.mutex.Unlock() + + r.withLogger(func(l *log.Logger) { + l.Printf("subscribed to partitions: %+v", offsetsByPartition) + }) + + return nil +} + +// connect returns a connection to ANY broker +func (r *Reader) connect() (conn *Conn, err error) { + for _, broker := range r.config.Brokers { + if conn, err = r.config.Dialer.Dial("tcp", broker); err == nil { + return + } + } + return // err will be non-nil +} + +// coordinator returns a connection to the coordinator for this group +func (r *Reader) coordinator() (*Conn, error) { + r.mutex.Lock() + address := r.address + r.mutex.Unlock() + + conn, err := r.config.Dialer.DialContext(r.stctx, "tcp", address) + if err != nil { + return nil, fmt.Errorf("unable to connect to coordinator, %v", address) + } + + return conn, nil +} + +func (r *Reader) waitThrottleTime(throttleTimeMS int32) { + if throttleTimeMS == 0 { + return + } + + t := time.NewTimer(time.Duration(throttleTimeMS) * time.Millisecond) + defer t.Stop() + + select { + case <-r.stctx.Done(): + return + case <-t.C: + } +} + +// heartbeat sends heartbeat to coordinator at the interval defined by +// ReaderConfig.HeartbeatInterval +func (r *Reader) heartbeat(conn *Conn) error { + generationID, memberID := r.membership() + if generationID == 0 && memberID == "" { + return nil + } + + resp, err := conn.heartbeat(heartbeatRequestV1{ + GroupID: r.config.GroupID, + GenerationID: generationID, + MemberID: memberID, + }) + if err != nil { + return fmt.Errorf("heartbeat failed: %v", err) + } + + r.waitThrottleTime(resp.ThrottleTimeMS) + + return nil +} + +func (r *Reader) heartbeatLoop(conn *Conn) func(stop <-chan struct{}) { + return func(stop <-chan struct{}) { + r.withLogger(func(l *log.Logger) { + l.Printf("started heartbeat for group, %v [%v]", r.config.GroupID, r.config.HeartbeatInterval) + }) + defer r.withLogger(func(l *log.Logger) { + l.Println("stopped heartbeat for group,", r.config.GroupID) + }) + + ticker := time.NewTicker(r.config.HeartbeatInterval) + defer ticker.Stop() + + for { + select { + case <-ticker.C: + if err := r.heartbeat(conn); err != nil { + return + } + + case <-stop: + return + } + } + } +} + +type offsetCommitter interface { + offsetCommit(request offsetCommitRequestV3) (offsetCommitResponseV3, error) +} + +func (r *Reader) commitOffsets(conn offsetCommitter, offsetStash offsetStash) error { + if len(offsetStash) == 0 { + return nil + } + + generationID, memberID := r.membership() + request := offsetCommitRequestV3{ + GroupID: r.config.GroupID, + GenerationID: generationID, + MemberID: memberID, + RetentionTime: int64(r.config.RetentionTime / time.Millisecond), + } + + for topic, partitions := range offsetStash { + t := offsetCommitRequestV3Topic{Topic: topic} + for partition, offset := range partitions { + t.Partitions = append(t.Partitions, offsetCommitRequestV3Partition{ + Partition: int32(partition), + Offset: offset, + }) + } + request.Topics = append(request.Topics, t) + } + + if _, err := conn.offsetCommit(request); err != nil { + return fmt.Errorf("unable to commit offsets for group, %v: %v", r.config.GroupID, err) + } + + r.withLogger(func(l *log.Logger) { + l.Printf("committed offsets: %v", offsetStash) + }) + + return nil +} + +// commitOffsetsWithRetry attempts to commit the specified offsets and retries +// up to the specified number of times +func (r *Reader) commitOffsetsWithRetry(conn offsetCommitter, offsetStash offsetStash, retries int) (err error) { + const ( + backoffDelayMin = 100 * time.Millisecond + backoffDelayMax = 5 * time.Second + ) + + for attempt := 0; attempt < retries; attempt++ { + if attempt != 0 { + if !sleep(r.stctx, backoff(attempt, backoffDelayMin, backoffDelayMax)) { + return + } + } + + if err = r.commitOffsets(conn, offsetStash); err == nil { + return + } + } + + return // err will not be nil +} + +// offsetStash holds offsets by topic => partition => offset +type offsetStash map[string]map[int]int64 + +// merge updates the offsetStash with the offsets from the provided messages +func (o offsetStash) merge(commits []commit) { + for _, c := range commits { + offsetsByPartition, ok := o[c.topic] + if !ok { + offsetsByPartition = map[int]int64{} + o[c.topic] = offsetsByPartition + } + + if offset, ok := offsetsByPartition[c.partition]; !ok || c.offset > offset { + offsetsByPartition[c.partition] = c.offset + } + } +} + +// reset clears the contents of the offsetStash +func (o offsetStash) reset() { + for key := range o { + delete(o, key) + } +} + +// commitLoopImmediate handles each commit synchronously +func (r *Reader) commitLoopImmediate(conn offsetCommitter, stop <-chan struct{}) { + offsetsByTopicAndPartition := offsetStash{} + + for { + select { + case <-stop: + return + + case req := <-r.commits: + offsetsByTopicAndPartition.merge(req.commits) + req.errch <- r.commitOffsetsWithRetry(conn, offsetsByTopicAndPartition, defaultCommitRetries) + offsetsByTopicAndPartition.reset() + } + } +} + +// commitLoopInterval handles each commit asynchronously with a period defined +// by ReaderConfig.CommitInterval +func (r *Reader) commitLoopInterval(conn offsetCommitter, stop <-chan struct{}) { + ticker := time.NewTicker(r.config.HeartbeatInterval) + defer ticker.Stop() + + commit := func() { + if err := r.commitOffsetsWithRetry(conn, r.offsetStash, defaultCommitRetries); err != nil { + r.withErrorLogger(func(l *log.Logger) { l.Print(err) }) + } else { + r.offsetStash.reset() + } + } + + for { + select { + case <-stop: + commit() + return + + case <-ticker.C: + commit() + + case req := <-r.commits: + r.offsetStash.merge(req.commits) + } + } +} + +// commitLoop processes commits off the commit chan +func (r *Reader) commitLoop(conn *Conn) func(stop <-chan struct{}) { + return func(stop <-chan struct{}) { + r.withLogger(func(l *log.Logger) { + l.Println("started commit for group,", r.config.GroupID) + }) + defer r.withLogger(func(l *log.Logger) { + l.Println("stopped commit for group,", r.config.GroupID) + }) + + if r.config.CommitInterval == 0 { + r.commitLoopImmediate(conn, stop) + } else { + r.commitLoopInterval(conn, stop) + } + } +} + +// handshake performs the necessary incantations to join this Reader to the desired +// consumer group. handshake will be called whenever the group is disrupted +// (member join, member leave, coordinator changed, etc) +func (r *Reader) handshake() error { + // always clear prior to subscribe + r.unsubscribe() + + // rebalance and fetch assignments + assignments, err := r.rebalance() + if err != nil { + return fmt.Errorf("rebalance failed for consumer group, %v: %v", r.config.GroupID, err) + } + + conn, err := r.coordinator() + if err != nil { + return fmt.Errorf("heartbeat: unable to connect to coordinator: %v", err) + } + defer conn.Close() + + rg := &runGroup{} + rg = rg.WithContext(r.stctx) + rg.Go(r.heartbeatLoop(conn)) + rg.Go(r.commitLoop(conn)) + + // subscribe to assignments + if err := r.subscribe(assignments); err != nil { + rg.Stop() + return fmt.Errorf("subscribe failed for consumer group, %v: %v\n", r.config.GroupID, err) + } + + rg.Wait() + + return nil +} + +// run provides the main consumer group management loop. Each iteration performs the +// handshake to join the Reader to the consumer group. +func (r *Reader) run() { + defer close(r.done) + + if !r.useConsumerGroup() { + return + } + + r.withLogger(func(l *log.Logger) { + l.Printf("entering loop for consumer group, %v\n", r.config.GroupID) + }) + + for { + if err := r.handshake(); err != nil { + r.withErrorLogger(func(l *log.Logger) { + l.Println(err) + }) + } + + select { + case <-r.stctx.Done(): + return + default: + } + } +} + +// ReaderConfig is a configuration object used to create new instances of +// Reader. +type ReaderConfig struct { + // The list of broker addresses used to connect to the kafka cluster. + Brokers []string + + // GroupID holds the optional consumer group id. If GroupID is specified, then + // Partition should NOT be specified e.g. 0 + GroupID string + + // The topic to read messages from. + Topic string + + // Partition to read messages from. Either Partition or GroupID may + // be assigned, but not both + Partition int + + // An dialer used to open connections to the kafka server. This field is + // optional, if nil, the default dialer is used instead. + Dialer *Dialer + + // The capacity of the internal message queue, defaults to 100 if none is + // set. + QueueCapacity int + + // Min and max number of bytes to fetch from kafka in each request. + MinBytes int + MaxBytes int + + // Maximum amount of time to wait for new data to come when fetching batches + // of messages from kafka. + MaxWait time.Duration + + // ReadLagInterval sets the frequency at which the reader lag is updated. + // Setting this field to a negative value disables lag reporting. + ReadLagInterval time.Duration + + // HeartbeatInterval sets the optional frequency at which the reader sends the consumer + // group heartbeat update. + // + // Default: 3s + // + // Only used when GroupID is set + HeartbeatInterval time.Duration + + // CommitInterval indicates the interval at which offsets are committed to + // the broker. If 0, commits will be handled synchronously. + // + // Defaults to 1s + // + // Only used when GroupID is set + CommitInterval time.Duration + + // SessionTimeout optionally sets the length of time that may pass without a heartbeat + // before the coordinator considers the consumer dead and initiates a rebalance. + // + // Default: 30s + // + // Only used when GroupID is set + SessionTimeout time.Duration + + // RebalanceTimeout optionally sets the length of time the coordinator will wait + // for members to join as part of a rebalance. For kafka servers under higher + // load, it may be useful to set this value higher. + // + // Default: 30s + // + // Only used when GroupID is set + RebalanceTimeout time.Duration + + // RetentionTime optionally sets the length of time the consumer group will be saved + // by the broker + // + // Default: 24h + // + // Only used when GroupID is set + RetentionTime time.Duration + + // If not nil, specifies a logger used to report internal changes within the + // reader. + Logger *log.Logger + + // ErrorLogger is the logger used to report errors. If nil, the reader falls + // back to using Logger instead. + ErrorLogger *log.Logger +} + +// ReaderStats is a data structure returned by a call to Reader.Stats that exposes +// details about the behavior of the reader. +type ReaderStats struct { + Dials int64 `metric:"kafka.reader.dial.count" type:"counter"` + Fetches int64 `metric:"kafak.reader.fetch.count" type:"counter"` // typo here, but I'm reluctant to fix it + Messages int64 `metric:"kafka.reader.message.count" type:"counter"` + Bytes int64 `metric:"kafka.reader.message.bytes" type:"counter"` + Rebalances int64 `metric:"kafka.reader.rebalance.count" type:"counter"` + Timeouts int64 `metric:"kafka.reader.timeout.count" type:"counter"` + Errors int64 `metric:"kafka.reader.error.count" type:"counter"` + + DialTime DurationStats `metric:"kafka.reader.dial.seconds"` + ReadTime DurationStats `metric:"kafka.reader.read.seconds"` + WaitTime DurationStats `metric:"kafka.reader.wait.seconds"` + FetchSize SummaryStats `metric:"kafka.reader.fetch.size"` + FetchBytes SummaryStats `metric:"kafka.reader.fetch.bytes"` + + Offset int64 `metric:"kafka.reader.offset" type:"gauge"` + Lag int64 `metric:"kafka.reader.lag" type:"gauge"` + MinBytes int64 `metric:"kafka.reader.fetch_bytes.min" type:"gauge"` + MaxBytes int64 `metric:"kafka.reader.fetch_bytes.max" type:"gauge"` + MaxWait time.Duration `metric:"kafka.reader.fetch_wait.max" type:"gauge"` + QueueLength int64 `metric:"kafka.reader.queue.length" type:"gauge"` + QueueCapacity int64 `metric:"kafka.reader.queue.capacity" type:"gauge"` + + ClientID string `tag:"client_id"` + Topic string `tag:"topic"` + Partition string `tag:"partition"` +} + +// readerStats is a struct that contains statistics on a reader. +type readerStats struct { + dials counter + fetches counter + messages counter + bytes counter + rebalances counter + timeouts counter + errors counter + dialTime summary + readTime summary + waitTime summary + fetchSize summary + fetchBytes summary + offset gauge + lag gauge + partition string +} + +// NewReader creates and returns a new Reader configured with config. +func NewReader(config ReaderConfig) *Reader { + if len(config.Brokers) == 0 { + panic("cannot create a new kafka reader with an empty list of broker addresses") + } + + if len(config.Topic) == 0 { + panic("cannot create a new kafka reader with an empty topic") + } + + if config.Partition < 0 || config.Partition >= math.MaxInt32 { + panic(fmt.Sprintf("partition number out of bounds: %d", config.Partition)) + } + + if config.MinBytes > config.MaxBytes { + panic(fmt.Sprintf("minimum batch size greater than the maximum (min = %d, max = %d)", config.MinBytes, config.MaxBytes)) + } + + if config.MinBytes < 0 { + panic(fmt.Sprintf("invalid negative minimum batch size (min = %d)", config.MinBytes)) + } + + if config.MaxBytes < 0 { + panic(fmt.Sprintf("invalid negative maximum batch size (max = %d)", config.MaxBytes)) + } + + if config.GroupID != "" && config.Partition != 0 { + panic("either Partition or GroupID may be specified, but not both") + } + + if config.GroupID != "" { + if config.HeartbeatInterval < 0 || (config.HeartbeatInterval/time.Millisecond) >= math.MaxInt32 { + panic(fmt.Sprintf("HeartbeatInterval out of bounds: %d", config.HeartbeatInterval)) + } + + if config.SessionTimeout < 0 || (config.SessionTimeout/time.Millisecond) >= math.MaxInt32 { + panic(fmt.Sprintf("SessionTimeout out of bounds: %d", config.SessionTimeout)) + } + + if config.RebalanceTimeout < 0 || (config.RebalanceTimeout/time.Millisecond) >= math.MaxInt32 { + panic(fmt.Sprintf("RebalanceTimeout out of bounds: %d", config.RebalanceTimeout)) + } + + if config.RetentionTime < 0 || (config.RetentionTime/time.Millisecond) >= math.MaxInt32 { + panic(fmt.Sprintf("RetentionTime out of bounds: %d", config.RetentionTime)) + } + + if config.CommitInterval < 0 || (config.CommitInterval/time.Millisecond) >= math.MaxInt32 { + panic(fmt.Sprintf("CommitInterval out of bounds: %d", config.CommitInterval)) + } + } + + if config.Dialer == nil { + config.Dialer = DefaultDialer + } + + if config.MaxBytes == 0 { + config.MaxBytes = 1e6 // 1 MB + } + + if config.MinBytes == 0 { + config.MinBytes = config.MaxBytes + } + + if config.MaxWait == 0 { + config.MaxWait = 10 * time.Second + } + + if config.ReadLagInterval == 0 { + config.ReadLagInterval = 1 * time.Minute + } + + if config.HeartbeatInterval == 0 { + config.HeartbeatInterval = defaultHeartbeatInterval + } + + if config.SessionTimeout == 0 { + config.SessionTimeout = defaultSessionTimeout + } + + if config.RebalanceTimeout == 0 { + config.RebalanceTimeout = defaultRebalanceTimeout + } + + if config.RetentionTime == 0 { + config.RetentionTime = defaultRetentionTime + } + + if config.QueueCapacity == 0 { + config.QueueCapacity = 100 + } + + // when configured as a consumer group; stats should report a partition of -1 + readerStatsPartition := config.Partition + if config.GroupID != "" { + readerStatsPartition = -1 + } + + // when configured as a consume group, start version as 1 to ensure that only + // the rebalance function will start readers + version := int64(0) + if config.GroupID != "" { + version = 1 + } + + stctx, stop := context.WithCancel(context.Background()) + r := &Reader{ + config: config, + msgs: make(chan readerMessage, config.QueueCapacity), + cancel: func() {}, + done: make(chan struct{}), + commits: make(chan commitRequest), + stop: stop, + offset: firstOffset, + stctx: stctx, + stats: &readerStats{ + dialTime: makeSummary(), + readTime: makeSummary(), + waitTime: makeSummary(), + fetchSize: makeSummary(), + fetchBytes: makeSummary(), + // Generate the string representation of the partition number only + // once when the reader is created. + partition: strconv.Itoa(readerStatsPartition), + }, + version: version, + offsetStash: offsetStash{}, + } + + go r.run() + + return r +} + +// Config returns the reader's configuration. +func (r *Reader) Config() ReaderConfig { + return r.config +} + +// Close closes the stream, preventing the program from reading any more +// messages from it. +func (r *Reader) Close() error { + atomic.StoreUint32(&r.once, 1) + + r.mutex.Lock() + closed := r.closed + r.closed = true + r.mutex.Unlock() + + r.cancel() + r.stop() + r.join.Wait() + + if r.useConsumerGroup() { + // gracefully attempt to leave the consumer group on close + if generationID, membershipID := r.membership(); generationID > 0 && membershipID != "" { + if conn, err := r.coordinator(); err == nil { + _ = r.leaveGroup(conn) + } + } + } + + <-r.done + + if !closed { + close(r.msgs) + } + + return nil +} + +// ReadMessage reads and return the next message from the r. The method call +// blocks until a message becomes available, or an error occurs. The program +// may also specify a context to asynchronously cancel the blocking operation. +// +// The method returns io.EOF to indicate that the reader has been closed. +// +// If consumer groups are used, ReadMessage will automatically commit the +// offset when called. +func (r *Reader) ReadMessage(ctx context.Context) (Message, error) { + m, err := r.FetchMessage(ctx) + if err != nil { + return Message{}, err + } + + if r.useConsumerGroup() { + if err := r.CommitMessages(ctx, m); err != nil { + return Message{}, err + } + } + + return m, nil +} + +// FetchMessage reads and return the next message from the r. The method call +// blocks until a message becomes available, or an error occurs. The program +// may also specify a context to asynchronously cancel the blocking operation. +// +// The method returns io.EOF to indicate that the reader has been closed. +// +// FetchMessage does not commit offsets automatically when using consumer groups. +// Use CommitMessages to commit the offset. +func (r *Reader) FetchMessage(ctx context.Context) (Message, error) { + r.activateReadLag() + + for { + r.mutex.Lock() + + if !r.closed && r.version == 0 { + r.start(map[int]int64{r.config.Partition: r.offset}) + } + + version := r.version + r.mutex.Unlock() + + select { + case <-ctx.Done(): + return Message{}, ctx.Err() + + case m, ok := <-r.msgs: + if !ok { + return Message{}, io.EOF + } + + if m.version >= version { + r.mutex.Lock() + + switch { + case m.error != nil: + case version == r.version: + r.offset = m.message.Offset + 1 + r.lag = m.watermark - r.offset + } + + r.mutex.Unlock() + + switch m.error { + case nil: + case io.EOF: + // io.EOF is used as a marker to indicate that the stream + // has been closed, in case it was received from the inner + // reader we don't want to confuse the program and replace + // the error with io.ErrUnexpectedEOF. + m.error = io.ErrUnexpectedEOF + } + + return m.message, m.error + } + } + } +} + +// CommitMessages commits the list of messages passed as argument. The program +// may pass a context to asynchronously cancel the commit operation when it was +// configured to be blocking. +func (r *Reader) CommitMessages(ctx context.Context, msgs ...Message) error { + if !r.useConsumerGroup() { + return errOnlyAvailableWithGroup + } + + var errch <-chan error + var sync = r.useSyncCommits() + var creq = commitRequest{ + commits: makeCommits(msgs...), + } + + if sync { + ch := make(chan error, 1) + errch, creq.errch = ch, ch + } + + select { + case r.commits <- creq: + case <-ctx.Done(): + return ctx.Err() + case <-r.stctx.Done(): + // This context is used to ensure we don't allow commits after the + // reader was closed. + return io.ErrClosedPipe + } + + if !sync { + return nil + } + + select { + case <-ctx.Done(): + return ctx.Err() + case err := <-errch: + return err + } +} + +// ReadLag returns the current lag of the reader by fetching the last offset of +// the topic and partition and computing the difference between that value and +// the offset of the last message returned by ReadMessage. +// +// This method is intended to be used in cases where a program may be unable to +// call ReadMessage to update the value returned by Lag, but still needs to get +// an up to date estimation of how far behind the reader is. For example when +// the consumer is not ready to process the next message. +// +// The function returns a lag of zero when the reader's current offset is +// negative. +func (r *Reader) ReadLag(ctx context.Context) (lag int64, err error) { + if r.useConsumerGroup() { + return 0, errNotAvailableWithGroup + } + + type offsets struct { + first int64 + last int64 + } + + offch := make(chan offsets, 1) + errch := make(chan error, 1) + + go func() { + var off offsets + var err error + + for _, broker := range r.config.Brokers { + var conn *Conn + + if conn, err = r.config.Dialer.DialLeader(ctx, "tcp", broker, r.config.Topic, r.config.Partition); err != nil { + continue + } + + deadline, _ := ctx.Deadline() + conn.SetDeadline(deadline) + + off.first, off.last, err = conn.ReadOffsets() + conn.Close() + + if err == nil { + break + } + } + + if err != nil { + errch <- err + } else { + offch <- off + } + }() + + select { + case off := <-offch: + switch cur := r.Offset(); { + case cur == firstOffset: + lag = off.last - off.first + + case cur == lastOffset: + lag = 0 + + default: + lag = off.last - cur + } + case err = <-errch: + case <-ctx.Done(): + err = ctx.Err() + } + + return +} + +// Offset returns the current offset of the reader. +func (r *Reader) Offset() int64 { + if r.useConsumerGroup() { + return -1 + } + + r.mutex.Lock() + offset := r.offset + r.mutex.Unlock() + r.withLogger(func(log *log.Logger) { + log.Printf("looking up offset of kafka reader for partition %d of %s: %d", r.config.Partition, r.config.Topic, offset) + }) + return offset +} + +// Lag returns the lag of the last message returned by ReadMessage. +func (r *Reader) Lag() int64 { + if r.useConsumerGroup() { + return -1 + } + + r.mutex.Lock() + lag := r.lag + r.mutex.Unlock() + return lag +} + +// SetOffset changes the offset from which the next batch of messages will be +// read. +// +// Setting the offset ot -1 means to seek to the first offset. +// Setting the offset to -2 means to seek to the last offset. +// +// The method fails with io.ErrClosedPipe if the reader has already been closed. +func (r *Reader) SetOffset(offset int64) error { + if r.useConsumerGroup() { + return errNotAvailableWithGroup + } + + var err error + r.mutex.Lock() + + if r.closed { + err = io.ErrClosedPipe + } else if offset != r.offset { + r.withLogger(func(log *log.Logger) { + log.Printf("setting the offset of the kafka reader for partition %d of %s from %d to %d", + r.config.Partition, r.config.Topic, r.offset, offset) + }) + r.offset = offset + + if r.version != 0 { + r.start(map[int]int64{r.config.Partition: r.offset}) + } + + r.activateReadLag() + } + + r.mutex.Unlock() + return err +} + +// Stats returns a snapshot of the reader stats since the last time the method +// was called, or since the reader was created if it is called for the first +// time. +// +// A typical use of this method is to spawn a goroutine that will periodically +// call Stats on a kafka reader and report the metrics to a stats collection +// system. +func (r *Reader) Stats() ReaderStats { + return ReaderStats{ + Dials: r.stats.dials.snapshot(), + Fetches: r.stats.fetches.snapshot(), + Messages: r.stats.messages.snapshot(), + Bytes: r.stats.bytes.snapshot(), + Rebalances: r.stats.rebalances.snapshot(), + Timeouts: r.stats.timeouts.snapshot(), + Errors: r.stats.errors.snapshot(), + DialTime: r.stats.dialTime.snapshotDuration(), + ReadTime: r.stats.readTime.snapshotDuration(), + WaitTime: r.stats.waitTime.snapshotDuration(), + FetchSize: r.stats.fetchSize.snapshot(), + FetchBytes: r.stats.fetchBytes.snapshot(), + Offset: r.stats.offset.snapshot(), + Lag: r.stats.lag.snapshot(), + MinBytes: int64(r.config.MinBytes), + MaxBytes: int64(r.config.MaxBytes), + MaxWait: r.config.MaxWait, + QueueLength: int64(len(r.msgs)), + QueueCapacity: int64(cap(r.msgs)), + ClientID: r.config.Dialer.ClientID, + Topic: r.config.Topic, + Partition: r.stats.partition, + } +} + +func (r *Reader) withLogger(do func(*log.Logger)) { + if r.config.Logger != nil { + do(r.config.Logger) + } +} + +func (r *Reader) withErrorLogger(do func(*log.Logger)) { + if r.config.ErrorLogger != nil { + do(r.config.ErrorLogger) + } else { + r.withLogger(do) + } +} + +func (r *Reader) activateReadLag() { + if r.config.ReadLagInterval > 0 && atomic.CompareAndSwapUint32(&r.once, 0, 1) { + // read lag will only be calculated when not using consumer groups + // todo discuss how capturing read lag should interact with rebalancing + if !r.useConsumerGroup() { + go r.readLag(r.stctx) + } + } +} + +func (r *Reader) readLag(ctx context.Context) { + ticker := time.NewTicker(r.config.ReadLagInterval) + defer ticker.Stop() + + for { + timeout, cancel := context.WithTimeout(ctx, r.config.ReadLagInterval/2) + lag, err := r.ReadLag(timeout) + cancel() + + if err != nil { + r.stats.errors.observe(1) + r.withErrorLogger(func(log *log.Logger) { + log.Printf("kafka reader failed to read lag of partition %d of %s", r.config.Partition, r.config.Topic) + }) + } else { + r.stats.lag.observe(lag) + } + + select { + case <-ticker.C: + case <-ctx.Done(): + return + } + } +} + +func (r *Reader) start(offsetsByPartition map[int]int64) { + if r.closed { + // don't start child reader if parent Reader is closed + return + } + + ctx, cancel := context.WithCancel(context.Background()) + + r.cancel() // always cancel the previous reader + r.cancel = cancel + r.version++ + + r.join.Add(len(offsetsByPartition)) + for partition, offset := range offsetsByPartition { + go func(ctx context.Context, partition int, offset int64, join *sync.WaitGroup) { + defer join.Done() + + (&reader{ + dialer: r.config.Dialer, + logger: r.config.Logger, + errorLogger: r.config.ErrorLogger, + brokers: r.config.Brokers, + topic: r.config.Topic, + partition: partition, + minBytes: r.config.MinBytes, + maxBytes: r.config.MaxBytes, + maxWait: r.config.MaxWait, + version: r.version, + msgs: r.msgs, + stats: r.stats, + }).run(ctx, offset) + }(ctx, partition, offset, &r.join) + } +} + +// A reader reads messages from kafka and produces them on its channels, it's +// used as an way to asynchronously fetch messages while the main program reads +// them using the high level reader API. +type reader struct { + dialer *Dialer + logger *log.Logger + errorLogger *log.Logger + brokers []string + topic string + partition int + minBytes int + maxBytes int + maxWait time.Duration + version int64 + msgs chan<- readerMessage + stats *readerStats +} + +type readerMessage struct { + version int64 + message Message + watermark int64 + error error +} + +func (r *reader) run(ctx context.Context, offset int64) { + const backoffDelayMin = 100 * time.Millisecond + const backoffDelayMax = 1 * time.Second + + // This is the reader's main loop, it only ends if the context is canceled + // and will keep attempting to reader messages otherwise. + // + // Retrying indefinitely has the nice side effect of preventing Read calls + // on the parent reader to block if connection to the kafka server fails, + // the reader keeps reporting errors on the error channel which will then + // be surfaced to the program. + // If the reader wasn't retrying then the program would block indefinitely + // on a Read call after reading the first error. + for attempt := 0; true; attempt++ { + if attempt != 0 { + if !sleep(ctx, backoff(attempt, backoffDelayMin, backoffDelayMax)) { + return + } + } + + r.withLogger(func(log *log.Logger) { + log.Printf("initializing kafka reader for partition %d of %s starting at offset %d", r.partition, r.topic, offset) + }) + + conn, start, err := r.initialize(ctx, offset) + switch err { + case nil: + case OffsetOutOfRange: + // This would happen if the requested offset is passed the last + // offset on the partition leader. In that case we're just going + // to retry later hoping that enough data has been produced. + r.withErrorLogger(func(log *log.Logger) { + log.Printf("error initializing the kafka reader for partition %d of %s: %s", r.partition, r.topic, OffsetOutOfRange) + }) + continue + default: + // Wait 4 attempts before reporting the first errors, this helps + // mitigate situations where the kafka server is temporarily + // unavailable. + if attempt >= 3 { + r.sendError(ctx, err) + } else { + r.stats.errors.observe(1) + r.withErrorLogger(func(log *log.Logger) { + log.Printf("error initializing the kafka reader for partition %d of %s: %s", r.partition, r.topic, err) + }) + } + continue + } + + // Resetting the attempt counter ensures that if a failure occurs after + // a successful initialization we don't keep increasing the backoff + // timeout. + attempt = 0 + + // Now we're sure to have an absolute offset number, may anything happen + // to the connection we know we'll want to restart from this offset. + offset = start + + errcount := 0 + readLoop: + for { + if !sleep(ctx, backoff(errcount, backoffDelayMin, backoffDelayMax)) { + conn.Close() + return + } + + switch offset, err = r.read(ctx, offset, conn); err { + case nil: + errcount = 0 + + case NotLeaderForPartition: + r.withErrorLogger(func(log *log.Logger) { + log.Printf("failed to read from current broker for partition %d of %s at offset %d, not the leader", r.partition, r.topic, offset) + }) + + conn.Close() + + // The next call to .initialize will re-establish a connection to the proper + // partition leader. + r.stats.rebalances.observe(1) + break readLoop + + case RequestTimedOut: + // Timeout on the kafka side, this can be safely retried. + errcount = 0 + r.withErrorLogger(func(log *log.Logger) { + log.Printf("no messages received from kafka within the allocated time for partition %d of %s at offset %d", r.partition, r.topic, offset) + }) + r.stats.timeouts.observe(1) + continue + + case OffsetOutOfRange: + first, last, err := r.readOffsets(conn) + + if err != nil { + r.withErrorLogger(func(log *log.Logger) { + log.Printf("the kafka reader got an error while attempting to determine whether it was reading before the first offset or after the last offset of partition %d of %s: %s", r.partition, r.topic, err) + }) + conn.Close() + break readLoop + } + + switch { + case offset < first: + r.withErrorLogger(func(log *log.Logger) { + log.Printf("the kafka reader is reading before the first offset for partition %d of %s, skipping from offset %d to %d (%d messages)", r.partition, r.topic, offset, first, first-offset) + }) + offset, errcount = first, 0 + continue // retry immediately so we don't keep falling behind due to the backoff + + case offset < last: + errcount = 0 + continue // more messages have already become available, retry immediately + + default: + // We may be reading past the last offset, will retry later. + r.withErrorLogger(func(log *log.Logger) { + log.Printf("the kafka reader is reading passed the last offset for partition %d of %s at offset %d", r.partition, r.topic, offset) + }) + } + + case context.Canceled: + // Another reader has taken over, we can safely quit. + conn.Close() + return + + default: + if _, ok := err.(Error); ok { + r.sendError(ctx, err) + } else { + r.withErrorLogger(func(log *log.Logger) { + log.Printf("the kafka reader got an unknown error reading partition %d of %s at offset %d: %s", r.partition, r.topic, offset, err) + }) + r.stats.errors.observe(1) + conn.Close() + break readLoop + } + } + + errcount++ + } + } +} + +func (r *reader) initialize(ctx context.Context, offset int64) (conn *Conn, start int64, err error) { + for i := 0; i != len(r.brokers) && conn == nil; i++ { + var broker = r.brokers[i] + var first int64 + var last int64 + + t0 := time.Now() + conn, err = r.dialer.DialLeader(ctx, "tcp", broker, r.topic, r.partition) + t1 := time.Now() + r.stats.dials.observe(1) + r.stats.dialTime.observeDuration(t1.Sub(t0)) + + if err != nil { + continue + } + + if first, last, err = r.readOffsets(conn); err != nil { + conn.Close() + conn = nil + break + } + + switch { + case offset == firstOffset: + offset = first + + case offset == lastOffset: + offset = last + + case offset < first: + offset = first + } + + r.withLogger(func(log *log.Logger) { + log.Printf("the kafka reader for partition %d of %s is seeking to offset %d", r.partition, r.topic, offset) + }) + + if start, err = conn.Seek(offset, 1); err != nil { + conn.Close() + conn = nil + break + } + + conn.SetDeadline(time.Time{}) + } + + return +} + +func (r *reader) read(ctx context.Context, offset int64, conn *Conn) (int64, error) { + r.stats.fetches.observe(1) + r.stats.offset.observe(offset) + + t0 := time.Now() + conn.SetReadDeadline(t0.Add(r.maxWait)) + + batch := conn.ReadBatch(r.minBytes, r.maxBytes) + highWaterMark := batch.HighWaterMark() + + t1 := time.Now() + r.stats.waitTime.observeDuration(t1.Sub(t0)) + + var msg Message + var err error + var size int64 + var bytes int64 + + const safetyTimeout = 10 * time.Second + deadline := time.Now().Add(safetyTimeout) + conn.SetReadDeadline(deadline) + + for { + if now := time.Now(); deadline.Sub(now) < (safetyTimeout / 2) { + deadline = now.Add(safetyTimeout) + conn.SetReadDeadline(deadline) + } + + if msg, err = batch.ReadMessage(); err != nil { + err = batch.Close() + break + } + + n := int64(len(msg.Key) + len(msg.Value)) + r.stats.messages.observe(1) + r.stats.bytes.observe(n) + + if err = r.sendMessage(ctx, msg, highWaterMark); err != nil { + err = batch.Close() + break + } + + offset = msg.Offset + 1 + r.stats.offset.observe(offset) + r.stats.lag.observe(highWaterMark - offset) + + size++ + bytes += n + } + + conn.SetReadDeadline(time.Time{}) + + t2 := time.Now() + r.stats.readTime.observeDuration(t2.Sub(t1)) + r.stats.fetchSize.observe(size) + r.stats.fetchBytes.observe(bytes) + return offset, err +} + +func (r *reader) readOffsets(conn *Conn) (first int64, last int64, err error) { + conn.SetDeadline(time.Now().Add(10 * time.Second)) + return conn.ReadOffsets() +} + +func (r *reader) sendMessage(ctx context.Context, msg Message, watermark int64) error { + select { + case r.msgs <- readerMessage{version: r.version, message: msg, watermark: watermark}: + return nil + case <-ctx.Done(): + return ctx.Err() + } +} + +func (r *reader) sendError(ctx context.Context, err error) error { + select { + case r.msgs <- readerMessage{version: r.version, error: err}: + return nil + case <-ctx.Done(): + return ctx.Err() + } +} + +func (r *reader) withLogger(do func(*log.Logger)) { + if r.logger != nil { + do(r.logger) + } +} + +func (r *reader) withErrorLogger(do func(*log.Logger)) { + if r.errorLogger != nil { + do(r.errorLogger) + } else { + r.withLogger(do) + } +} + +// extractTopics returns the unique list of topics represented by the set of +// provided members +func extractTopics(members []memberGroupMetadata) []string { + var visited = map[string]struct{}{} + var topics []string + + for _, member := range members { + for _, topic := range member.Metadata.Topics { + if _, seen := visited[topic]; seen { + continue + } + + topics = append(topics, topic) + visited[topic] = struct{}{} + } + } + + sort.Strings(topics) + + return topics +} diff --git a/vendor/github.com/segmentio/kafka-go/rungroup.go b/vendor/github.com/segmentio/kafka-go/rungroup.go new file mode 100644 index 000000000..b8cd704f0 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/rungroup.go @@ -0,0 +1,61 @@ +package kafka + +import ( + "context" + "sync" +) + +// runGroup is a collection of goroutines working together. If any one goroutine +// stops, then all goroutines will be stopped. +// +// A zero runGroup is valid +type runGroup struct { + initOnce sync.Once + + ctx context.Context + cancel context.CancelFunc + + wg sync.WaitGroup +} + +func (r *runGroup) init() { + if r.cancel == nil { + r.ctx, r.cancel = context.WithCancel(context.Background()) + } +} + +func (r *runGroup) WithContext(ctx context.Context) *runGroup { + ctx, cancel := context.WithCancel(ctx) + return &runGroup{ + ctx: ctx, + cancel: cancel, + } +} + +// Wait blocks until all function calls have returned. +func (r *runGroup) Wait() { + r.wg.Wait() +} + +// Stop stops the goroutines and waits for them to complete +func (r *runGroup) Stop() { + r.initOnce.Do(r.init) + r.cancel() + r.Wait() +} + +// Go calls the given function in a new goroutine. +// +// The first call to return a non-nil error cancels the group; its error will be +// returned by Wait. +func (r *runGroup) Go(f func(stop <-chan struct{})) { + r.initOnce.Do(r.init) + + r.wg.Add(1) + go func() { + defer r.wg.Done() + defer r.cancel() + + f(r.ctx.Done()) + }() +} diff --git a/vendor/github.com/segmentio/kafka-go/sizeof.go b/vendor/github.com/segmentio/kafka-go/sizeof.go new file mode 100644 index 000000000..87feb1a14 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/sizeof.go @@ -0,0 +1,73 @@ +package kafka + +import "fmt" + +type sizable interface { + size() int32 +} + +func sizeof(a interface{}) int32 { + switch v := a.(type) { + case int8: + return 1 + case int16: + return 2 + case int32: + return 4 + case int64: + return 8 + case string: + return sizeofString(v) + case bool: + return 1 + case []byte: + return sizeofBytes(v) + case sizable: + return v.size() + } + panic(fmt.Sprintf("unsupported type: %T", a)) +} + +func sizeofInt8(_ int8) int32 { + return 1 +} + +func sizeofInt16(_ int16) int32 { + return 2 +} + +func sizeofInt32(_ int32) int32 { + return 4 +} + +func sizeofInt64(_ int64) int32 { + return 8 +} + +func sizeofString(s string) int32 { + return 2 + int32(len(s)) +} + +func sizeofBool(_ bool) int32 { + return 1 +} + +func sizeofBytes(b []byte) int32 { + return 4 + int32(len(b)) +} + +func sizeofArray(n int, f func(int) int32) int32 { + s := int32(4) + for i := 0; i != n; i++ { + s += f(i) + } + return s +} + +func sizeofInt32Array(a []int32) int32 { + return 4 + (4 * int32(len(a))) +} + +func sizeofStringArray(a []string) int32 { + return sizeofArray(len(a), func(i int) int32 { return sizeofString(a[i]) }) +} diff --git a/vendor/github.com/segmentio/kafka-go/stats.go b/vendor/github.com/segmentio/kafka-go/stats.go new file mode 100644 index 000000000..9c69dc066 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/stats.go @@ -0,0 +1,186 @@ +package kafka + +import ( + "sync/atomic" + "time" +) + +// SummaryStats is a data structure that carries a summary of observed values. +// The average, minimum, and maximum are reported. +type SummaryStats struct { + Avg int64 `metric:"avg" type:"gauge"` + Min int64 `metric:"min" type:"gauge"` + Max int64 `metric:"max" type:"gauge"` +} + +// DurationStats is a data structure that carries a summary of observed duration +// values. The average, minimum, and maximum are reported. +type DurationStats struct { + Avg time.Duration `metric:"avg" type:"gauge"` + Min time.Duration `metric:"min" type:"gauge"` + Max time.Duration `metric:"max" type:"gauge"` +} + +// counter is an atomic incrementing counter which gets reset on snapshot. +// +// Since atomic is used to mutate the statistic the value must be 64-bit aligned. +// See https://golang.org/pkg/sync/atomic/#pkg-note-BUG +type counter int64 + +func (c *counter) ptr() *int64 { + return (*int64)(c) +} + +func (c *counter) observe(v int64) { + atomic.AddInt64(c.ptr(), v) +} + +func (c *counter) snapshot() int64 { + p := c.ptr() + v := atomic.LoadInt64(p) + atomic.AddInt64(p, -v) + return v +} + +// gauge is an atomic integer that may be set to any arbitrary value, the value +// does not change after a snapshot. +// +// Since atomic is used to mutate the statistic the value must be 64-bit aligned. +// See https://golang.org/pkg/sync/atomic/#pkg-note-BUG +type gauge int64 + +func (g *gauge) ptr() *int64 { + return (*int64)(g) +} + +func (g *gauge) observe(v int64) { + atomic.StoreInt64(g.ptr(), v) +} + +func (g *gauge) snapshot() int64 { + return atomic.LoadInt64(g.ptr()) +} + +// minimum is an atomic integral type that keeps track of the minimum of all +// values that it observed between snapshots. +// +// Since atomic is used to mutate the statistic the value must be 64-bit aligned. +// See https://golang.org/pkg/sync/atomic/#pkg-note-BUG +type minimum int64 + +func (m *minimum) ptr() *int64 { + return (*int64)(m) +} + +func (m *minimum) observe(v int64) { + for { + ptr := m.ptr() + min := atomic.LoadInt64(ptr) + + if min >= 0 && min <= v { + break + } + + if atomic.CompareAndSwapInt64(ptr, min, v) { + break + } + } +} + +func (m *minimum) snapshot() int64 { + p := m.ptr() + v := atomic.LoadInt64(p) + atomic.CompareAndSwapInt64(p, v, -1) + if v < 0 { + v = 0 + } + return v +} + +// maximum is an atomic integral type that keeps track of the maximum of all +// values that it observed between snapshots. +// +// Since atomic is used to mutate the statistic the value must be 64-bit aligned. +// See https://golang.org/pkg/sync/atomic/#pkg-note-BUG +type maximum int64 + +func (m *maximum) ptr() *int64 { + return (*int64)(m) +} + +func (m *maximum) observe(v int64) { + for { + ptr := m.ptr() + max := atomic.LoadInt64(ptr) + + if max >= 0 && max >= v { + break + } + + if atomic.CompareAndSwapInt64(ptr, max, v) { + break + } + } +} + +func (m *maximum) snapshot() int64 { + p := m.ptr() + v := atomic.LoadInt64(p) + atomic.CompareAndSwapInt64(p, v, -1) + if v < 0 { + v = 0 + } + return v +} + +type summary struct { + min minimum + max maximum + sum counter + count counter +} + +func makeSummary() summary { + return summary{ + min: -1, + max: -1, + } +} + +func (s *summary) observe(v int64) { + s.min.observe(v) + s.max.observe(v) + s.sum.observe(v) + s.count.observe(1) +} + +func (s *summary) observeDuration(v time.Duration) { + s.observe(int64(v)) +} + +func (s *summary) snapshot() SummaryStats { + avg := int64(0) + min := s.min.snapshot() + max := s.max.snapshot() + sum := s.sum.snapshot() + count := s.count.snapshot() + + if count != 0 { + avg = int64(float64(sum) / float64(count)) + } + + return SummaryStats{ + Avg: avg, + Min: min, + Max: max, + } +} + +func (s *summary) snapshotDuration() DurationStats { + summary := s.snapshot() + return DurationStats{ + Avg: time.Duration(summary.Avg), + Min: time.Duration(summary.Min), + Max: time.Duration(summary.Max), + } +} diff --git a/vendor/github.com/segmentio/kafka-go/strategy.go b/vendor/github.com/segmentio/kafka-go/strategy.go new file mode 100644 index 000000000..5237c1cba --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/strategy.go @@ -0,0 +1,187 @@ +package kafka + +import "sort" + +// strategy encapsulates the client side rebalancing logic +type strategy interface { + // ProtocolName of strategy + ProtocolName() string + + // ProtocolMetadata provides the strategy an opportunity to embed custom + // UserData into the metadata. + // + // Will be used by JoinGroup to begin the consumer group handshake. + // + // See https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-JoinGroupRequest + GroupMetadata(topics []string) (groupMetadata, error) + + // DefineMemberships returns which members will be consuming + // which topic partitions + AssignGroups(members []memberGroupMetadata, partitions []Partition) memberGroupAssignments +} + +var ( + // allStrategies the kafka-go Reader supports + allStrategies = []strategy{ + rangeStrategy{}, + roundrobinStrategy{}, + } +) + +// rangeStrategy groups consumers by partition +// +// Example: 5 partitions, 2 consumers +// C0: [0, 1, 2] +// C1: [3, 4] +// +// Example: 6 partitions, 3 consumers +// C0: [0, 1] +// C1: [2, 3] +// C2: [4, 5] +// +type rangeStrategy struct{} + +func (r rangeStrategy) ProtocolName() string { + return "range" +} + +func (r rangeStrategy) GroupMetadata(topics []string) (groupMetadata, error) { + return groupMetadata{ + Version: 1, + Topics: topics, + }, nil +} + +func (r rangeStrategy) AssignGroups(members []memberGroupMetadata, topicPartitions []Partition) memberGroupAssignments { + groupAssignments := memberGroupAssignments{} + membersByTopic := findMembersByTopic(members) + + for topic, members := range membersByTopic { + partitions := findPartitions(topic, topicPartitions) + partitionCount := len(partitions) + memberCount := len(members) + + rangeSize := partitionCount / memberCount + if partitionCount%memberCount != 0 { + rangeSize++ + } + + for memberIndex, member := range members { + assignmentsByTopic, ok := groupAssignments[member.MemberID] + if !ok { + assignmentsByTopic = map[string][]int32{} + groupAssignments[member.MemberID] = assignmentsByTopic + } + + for partitionIndex, partition := range partitions { + if (partitionIndex / rangeSize) == memberIndex { + assignmentsByTopic[topic] = append(assignmentsByTopic[topic], partition) + } + } + } + } + + return groupAssignments +} + +// roundrobinStrategy divides partitions evenly among consumers +// +// Example: 5 partitions, 2 consumers +// C0: [0, 2, 4] +// C1: [1, 3] +// +// Example: 6 partitions, 3 consumers +// C0: [0, 3] +// C1: [1, 4] +// C2: [2, 5] +// +type roundrobinStrategy struct{} + +func (r roundrobinStrategy) ProtocolName() string { + return "roundrobin" +} + +func (r roundrobinStrategy) GroupMetadata(topics []string) (groupMetadata, error) { + return groupMetadata{ + Version: 1, + Topics: topics, + }, nil +} + +func (r roundrobinStrategy) AssignGroups(members []memberGroupMetadata, topicPartitions []Partition) memberGroupAssignments { + groupAssignments := memberGroupAssignments{} + membersByTopic := findMembersByTopic(members) + for topic, members := range membersByTopic { + partitionIDs := findPartitions(topic, topicPartitions) + memberCount := len(members) + + for memberIndex, member := range members { + assignmentsByTopic, ok := groupAssignments[member.MemberID] + if !ok { + assignmentsByTopic = map[string][]int32{} + groupAssignments[member.MemberID] = assignmentsByTopic + } + + for partitionIndex, partition := range partitionIDs { + if (partitionIndex % memberCount) == memberIndex { + assignmentsByTopic[topic] = append(assignmentsByTopic[topic], partition) + } + } + } + } + + return groupAssignments +} + +// findPartitions extracts the partition ids associated with the topic from the +// list of Partitions provided +func findPartitions(topic string, partitions []Partition) []int32 { + var ids []int32 + for _, partition := range partitions { + if partition.Topic == topic { + ids = append(ids, int32(partition.ID)) + } + } + return ids +} + +// findMembersByTopic groups the memberGroupMetadata by topic +func findMembersByTopic(members []memberGroupMetadata) map[string][]memberGroupMetadata { + membersByTopic := map[string][]memberGroupMetadata{} + for _, member := range members { + for _, topic := range member.Metadata.Topics { + membersByTopic[topic] = append(membersByTopic[topic], member) + } + } + + // normalize ordering of members to enabling grouping across topics by partitions + // + // Want: + // C0 [T0/P0, T1/P0] + // C1 [T0/P1, T1/P1] + // + // Not: + // C0 [T0/P0, T1/P1] + // C1 [T0/P1, T1/P0] + // + // Even though the later is still round robin, the partitions are crossed + // + for _, members := range membersByTopic { + sort.Slice(members, func(i, j int) bool { + return members[i].MemberID < members[j].MemberID + }) + } + + return membersByTopic +} + +// findStrategy returns the strategy with the specified protocolName from the +// slice provided +func findStrategy(protocolName string, strategies []strategy) (strategy, bool) { + for _, strategy := range strategies { + if strategy.ProtocolName() == protocolName { + return strategy, true + } + } + return nil, false +} diff --git a/vendor/github.com/segmentio/kafka-go/syncgroup.go b/vendor/github.com/segmentio/kafka-go/syncgroup.go new file mode 100644 index 000000000..241a9ea9c --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/syncgroup.go @@ -0,0 +1,154 @@ +package kafka + +import ( + "bufio" + "bytes" +) + +// memberGroupAssignments holds MemberID => topic => partitions +type memberGroupAssignments map[string]map[string][]int32 + +type groupAssignment struct { + Version int16 + Topics map[string][]int32 + UserData []byte +} + +func (t groupAssignment) size() int32 { + sz := sizeofInt16(t.Version) + sizeofInt16(int16(len(t.Topics))) + + for topic, partitions := range t.Topics { + sz += sizeofString(topic) + sizeofInt32Array(partitions) + } + + return sz + sizeofBytes(t.UserData) +} + +func (t groupAssignment) writeTo(w *bufio.Writer) { + writeInt16(w, t.Version) + writeInt32(w, int32(len(t.Topics))) + + for topic, partitions := range t.Topics { + writeString(w, topic) + writeInt32Array(w, partitions) + } + + writeBytes(w, t.UserData) +} + +func (t *groupAssignment) readFrom(r *bufio.Reader, size int) (remain int, err error) { + // I came across this case when testing for compatibility with bsm/sarama-cluster. It + // appears in some cases, sarama-cluster can send a nil array entry. Admittedly, I + // didn't look too closely at it. + if size == 0 { + t.Topics = map[string][]int32{} + return 0, nil + } + + if remain, err = readInt16(r, size, &t.Version); err != nil { + return + } + if remain, err = readMapStringInt32(r, remain, &t.Topics); err != nil { + return + } + if remain, err = readBytes(r, remain, &t.UserData); err != nil { + return + } + + return +} + +func (t groupAssignment) bytes() []byte { + buf := bytes.NewBuffer(nil) + w := bufio.NewWriter(buf) + t.writeTo(w) + w.Flush() + return buf.Bytes() +} + +type syncGroupRequestGroupAssignmentV1 struct { + // MemberID assigned by the group coordinator + MemberID string + + // MemberAssignments holds client encoded assignments + // + // See consumer groups section of https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol + MemberAssignments []byte +} + +func (t syncGroupRequestGroupAssignmentV1) size() int32 { + return sizeofString(t.MemberID) + + sizeofBytes(t.MemberAssignments) +} + +func (t syncGroupRequestGroupAssignmentV1) writeTo(w *bufio.Writer) { + writeString(w, t.MemberID) + writeBytes(w, t.MemberAssignments) +} + +type syncGroupRequestV1 struct { + // GroupID holds the unique group identifier + GroupID string + + // GenerationID holds the generation of the group. + GenerationID int32 + + // MemberID assigned by the group coordinator + MemberID string + + GroupAssignments []syncGroupRequestGroupAssignmentV1 +} + +func (t syncGroupRequestV1) size() int32 { + return sizeofString(t.GroupID) + + sizeofInt32(t.GenerationID) + + sizeofString(t.MemberID) + + sizeofArray(len(t.GroupAssignments), func(i int) int32 { return t.GroupAssignments[i].size() }) +} + +func (t syncGroupRequestV1) writeTo(w *bufio.Writer) { + writeString(w, t.GroupID) + writeInt32(w, t.GenerationID) + writeString(w, t.MemberID) + writeArray(w, len(t.GroupAssignments), func(i int) { t.GroupAssignments[i].writeTo(w) }) +} + +type syncGroupResponseV1 struct { + // ThrottleTimeMS holds the duration in milliseconds for which the request + // was throttled due to quota violation (Zero if the request did not violate + // any quota) + ThrottleTimeMS int32 + + // ErrorCode holds response error code + ErrorCode int16 + + // MemberAssignments holds client encoded assignments + // + // See consumer groups section of https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol + MemberAssignments []byte +} + +func (t syncGroupResponseV1) size() int32 { + return sizeofInt32(t.ThrottleTimeMS) + + sizeofInt16(t.ErrorCode) + + sizeofBytes(t.MemberAssignments) +} + +func (t syncGroupResponseV1) writeTo(w *bufio.Writer) { + writeInt32(w, t.ThrottleTimeMS) + writeInt16(w, t.ErrorCode) + writeBytes(w, t.MemberAssignments) +} + +func (t *syncGroupResponseV1) readFrom(r *bufio.Reader, sz int) (remain int, err error) { + if remain, err = readInt32(r, sz, &t.ThrottleTimeMS); err != nil { + return + } + if remain, err = readInt16(r, remain, &t.ErrorCode); err != nil { + return + } + if remain, err = readBytes(r, remain, &t.MemberAssignments); err != nil { + return + } + return +} diff --git a/vendor/github.com/segmentio/kafka-go/time.go b/vendor/github.com/segmentio/kafka-go/time.go new file mode 100644 index 000000000..26f33afd0 --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/time.go @@ -0,0 +1,55 @@ +package kafka + +import ( + "math" + "time" +) + +const ( + maxTimeout = time.Duration(math.MaxInt32) * time.Millisecond + minTimeout = time.Duration(math.MinInt32) * time.Millisecond + defaultRTT = 1 * time.Second +) + +func timestamp(t time.Time) int64 { + if t.IsZero() { + return 0 + } + return t.UnixNano() / int64(time.Millisecond) +} + +func timestampToTime(t int64) time.Time { + return time.Unix(t/1000, (t%1000)*int64(time.Millisecond)) +} + +func duration(ms int32) time.Duration { + return time.Duration(ms) * time.Millisecond +} + +func milliseconds(d time.Duration) int32 { + switch { + case d > maxTimeout: + d = maxTimeout + case d < minTimeout: + d = minTimeout + } + return int32(d / time.Millisecond) +} + +func deadlineToTimeout(deadline time.Time, now time.Time) time.Duration { + if deadline.IsZero() { + return maxTimeout + } + return deadline.Sub(now) +} + +func adjustDeadlineForRTT(deadline time.Time, now time.Time, rtt time.Duration) time.Time { + if !deadline.IsZero() { + timeout := deadline.Sub(now) + if timeout < rtt { + rtt = timeout / 4 + } + deadline = deadline.Add(-rtt) + } + return deadline +} diff --git a/vendor/github.com/segmentio/kafka-go/write.go b/vendor/github.com/segmentio/kafka-go/write.go new file mode 100644 index 000000000..c20ed6b3c --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/write.go @@ -0,0 +1,248 @@ +package kafka + +import ( + "bufio" + "encoding/binary" + "fmt" + "time" +) + +type writable interface { + writeTo(*bufio.Writer) +} + +func writeInt8(w *bufio.Writer, i int8) { + w.WriteByte(byte(i)) +} + +func writeInt16(w *bufio.Writer, i int16) { + var b [2]byte + binary.BigEndian.PutUint16(b[:], uint16(i)) + w.WriteByte(b[0]) + w.WriteByte(b[1]) +} + +func writeInt32(w *bufio.Writer, i int32) { + var b [4]byte + binary.BigEndian.PutUint32(b[:], uint32(i)) + w.WriteByte(b[0]) + w.WriteByte(b[1]) + w.WriteByte(b[2]) + w.WriteByte(b[3]) +} + +func writeInt64(w *bufio.Writer, i int64) { + var b [8]byte + binary.BigEndian.PutUint64(b[:], uint64(i)) + w.WriteByte(b[0]) + w.WriteByte(b[1]) + w.WriteByte(b[2]) + w.WriteByte(b[3]) + w.WriteByte(b[4]) + w.WriteByte(b[5]) + w.WriteByte(b[6]) + w.WriteByte(b[7]) +} + +func writeString(w *bufio.Writer, s string) { + writeInt16(w, int16(len(s))) + w.WriteString(s) +} + +func writeBytes(w *bufio.Writer, b []byte) { + n := len(b) + if b == nil { + n = -1 + } + writeInt32(w, int32(n)) + w.Write(b) +} + +func writeBool(w *bufio.Writer, b bool) { + v := int8(0) + if b { + v = 1 + } + writeInt8(w, v) +} + +func writeArrayLen(w *bufio.Writer, n int) { + writeInt32(w, int32(n)) +} + +func writeArray(w *bufio.Writer, n int, f func(int)) { + writeArrayLen(w, n) + for i := 0; i != n; i++ { + f(i) + } +} + +func writeStringArray(w *bufio.Writer, a []string) { + writeArray(w, len(a), func(i int) { writeString(w, a[i]) }) +} + +func writeInt32Array(w *bufio.Writer, a []int32) { + writeArray(w, len(a), func(i int) { writeInt32(w, a[i]) }) +} + +func write(w *bufio.Writer, a interface{}) { + switch v := a.(type) { + case int8: + writeInt8(w, v) + case int16: + writeInt16(w, v) + case int32: + writeInt32(w, v) + case int64: + writeInt64(w, v) + case string: + writeString(w, v) + case []byte: + writeBytes(w, v) + case bool: + writeBool(w, v) + case writable: + v.writeTo(w) + default: + panic(fmt.Sprintf("unsupported type: %T", a)) + } +} + +// The functions bellow are used as optimizations to avoid dynamic memory +// allocations that occur when building the data structures representing the +// kafka protocol requests. + +func writeFetchRequestV1(w *bufio.Writer, correlationID int32, clientID string, topic string, partition int32, offset int64, minBytes int, maxBytes int, maxWait time.Duration) error { + h := requestHeader{ + ApiKey: int16(fetchRequest), + ApiVersion: int16(v1), + CorrelationID: correlationID, + ClientID: clientID, + } + h.Size = (h.size() - 4) + + 4 + // replica ID + 4 + // max wait time + 4 + // min bytes + 4 + // topic array length + sizeofString(topic) + + 4 + // partition array length + 4 + // partition + 8 + // offset + 4 // max bytes + + h.writeTo(w) + writeInt32(w, -1) // replica ID + writeInt32(w, milliseconds(maxWait)) + writeInt32(w, int32(minBytes)) + + // topic array + writeArrayLen(w, 1) + writeString(w, topic) + + // partition array + writeArrayLen(w, 1) + writeInt32(w, partition) + writeInt64(w, offset) + writeInt32(w, int32(maxBytes)) + + return w.Flush() +} + +func writeListOffsetRequestV1(w *bufio.Writer, correlationID int32, clientID string, topic string, partition int32, time int64) error { + h := requestHeader{ + ApiKey: int16(listOffsetRequest), + ApiVersion: int16(v1), + CorrelationID: correlationID, + ClientID: clientID, + } + h.Size = (h.size() - 4) + + 4 + // replica ID + 4 + // topic array length + sizeofString(topic) + // topic + 4 + // partition array length + 4 + // partition + 8 // time + + h.writeTo(w) + writeInt32(w, -1) // replica ID + + // topic array + writeArrayLen(w, 1) + writeString(w, topic) + + // partition array + writeArrayLen(w, 1) + writeInt32(w, partition) + writeInt64(w, time) + + return w.Flush() +} + +func writeProduceRequestV2(w *bufio.Writer, correlationID int32, clientID string, topic string, partition int32, timeout time.Duration, requiredAcks int16, msgs ...Message) error { + var size int32 + + for _, msg := range msgs { + size += 8 + // offset + 4 + // message size + 4 + // crc + 1 + // magic byte + 1 + // attributes + 8 + // timestamp + sizeofBytes(msg.Key) + + sizeofBytes(msg.Value) + } + + h := requestHeader{ + ApiKey: int16(produceRequest), + ApiVersion: int16(v2), + CorrelationID: correlationID, + ClientID: clientID, + } + h.Size = (h.size() - 4) + + 2 + // required acks + 4 + // timeout + 4 + // topic array length + sizeofString(topic) + // topic + 4 + // partition array length + 4 + // partition + 4 + // message set size + size + + h.writeTo(w) + writeInt16(w, requiredAcks) // required acks + writeInt32(w, milliseconds(timeout)) + + // topic array + writeArrayLen(w, 1) + writeString(w, topic) + + // partition array + writeArrayLen(w, 1) + writeInt32(w, partition) + writeInt32(w, size) + + const magicByte = 1 + const attributes = 0 + + for _, msg := range msgs { + timestamp := timestamp(msg.Time) + crc32 := crc32OfMessage(magicByte, attributes, timestamp, msg.Key, msg.Value) + size := 4 + // crc + 1 + // magic byte + 1 + // attributes + 8 + // timestamp + sizeofBytes(msg.Key) + + sizeofBytes(msg.Value) + + writeInt64(w, msg.Offset) + writeInt32(w, int32(size)) + writeInt32(w, int32(crc32)) + writeInt8(w, magicByte) + writeInt8(w, attributes) + writeInt64(w, timestamp) + writeBytes(w, msg.Key) + writeBytes(w, msg.Value) + } + + return w.Flush() +} diff --git a/vendor/github.com/segmentio/kafka-go/writer.go b/vendor/github.com/segmentio/kafka-go/writer.go new file mode 100644 index 000000000..375ee5e0c --- /dev/null +++ b/vendor/github.com/segmentio/kafka-go/writer.go @@ -0,0 +1,695 @@ +package kafka + +import ( + "context" + "fmt" + "io" + "math/rand" + "sort" + "sync" + "time" +) + +// The Writer type provides the implementation of a producer of kafka messages +// that automatically distributes messages across partitions of a single topic +// using a configurable balancing policy. +// +// Instances of Writer are safe to use concurrently from multiple goroutines. +type Writer struct { + config WriterConfig + + mutex sync.RWMutex + closed bool + + join sync.WaitGroup + msgs chan writerMessage + done chan struct{} + + // writer stats are all made of atomic values, no need for synchronization. + // Use a pointer to ensure 64-bit alignment of the values. + stats *writerStats +} + +// WriterConfig is a configuration type used to create new instances of Writer. +type WriterConfig struct { + // The list of brokers used to discover the partitions available on the + // kafka cluster. + // + // This field is required, attempting to create a writer with an empty list + // of brokers will panic. + Brokers []string + + // The topic that the writer will produce messages to. + // + // This field is required, attempting to create a writer with an empty topic + // will panic. + Topic string + + // The dialer used by the writer to establish connections to the kafka + // cluster. + // + // If nil, the default dialer is used instead. + Dialer *Dialer + + // The balancer used to distribute messages across partitions. + // + // The default is to use a round-robin distribution. + Balancer Balancer + + // Limit on how many attempts will be made to deliver a message. + // + // The default is to try at most 10 times. + MaxAttempts int + + // A hint on the capacity of the writer's internal message queue. + // + // The default is to use a queue capacity of 100 messages. + QueueCapacity int + + // Limit on how many messages will be buffered before being sent to a + // partition. + // + // The default is to use a target batch size of 100 messages. + BatchSize int + + // Time limit on how often incomplete message batches will be flushed to + // kafka. + // + // The default is to flush at least every second. + BatchTimeout time.Duration + + // Timeout for read operations performed by the Writer. + // + // Defaults to 10 seconds. + ReadTimeout time.Duration + + // Timeout for write operation performed by the Writer. + // + // Defaults to 10 seconds. + WriteTimeout time.Duration + + // This interval defines how often the list of partitions is refreshed from + // kafka. It allows the writer to automatically handle when new partitions + // are added to a topic. + // + // The default is to refresh partitions every 15 seconds. + RebalanceInterval time.Duration + + // Number of acknowledges from partition replicas required before receiving + // a response to a produce request (default to -1, which means to wait for + // all replicas). + RequiredAcks int + + // Setting this flag to true causes the WriteMessages method to never block. + // It also means that errors are ignored since the caller will not receive + // the returned value. Use this only if you don't care about guarantees of + // whether the messages were written to kafka. + Async bool + + newPartitionWriter func(partition int, config WriterConfig, stats *writerStats) partitionWriter +} + +// WriterStats is a data structure returned by a call to Writer.Stats that +// exposes details about the behavior of the writer. +type WriterStats struct { + Dials int64 `metric:"kafka.writer.dial.count" type:"counter"` + Writes int64 `metric:"kafka.writer.write.count" type:"counter"` + Messages int64 `metric:"kafka.writer.message.count" type:"counter"` + Bytes int64 `metric:"kafka.writer.message.bytes" type:"counter"` + Rebalances int64 `metric:"kafka.writer.rebalance.count" type:"counter"` + Errors int64 `metric:"kafka.writer.error.count" type:"counter"` + + DialTime DurationStats `metric:"kafka.writer.dial.seconds"` + WriteTime DurationStats `metric:"kafka.writer.write.seconds"` + WaitTime DurationStats `metric:"kafka.writer.wait.seconds"` + Retries SummaryStats `metric:"kafka.writer.retries.count"` + BatchSize SummaryStats `metric:"kafka.writer.batch.size"` + + MaxAttempts int64 `metric:"kafka.writer.attempts.max" type:"gauge"` + MaxBatchSize int64 `metric:"kafka.writer.batch.max" type:"gauge"` + BatchTimeout time.Duration `metric:"kafka.writer.batch.timeout" type:"gauge"` + ReadTimeout time.Duration `metric:"kafka.writer.read.timeout" type:"gauge"` + WriteTimeout time.Duration `metric:"kafka.writer.write.timeout" type:"gauge"` + RebalanceInterval time.Duration `metric:"kafka.writer.rebalance.interval" type:"gauge"` + RequiredAcks int64 `metric:"kafka.writer.acks.required" type:"gauge"` + Async bool `metric:"kafka.writer.async" type:"gauge"` + QueueLength int64 `metric:"kafka.writer.queue.length" type:"gauge"` + QueueCapacity int64 `metric:"kafka.writer.queue.capacity" type:"gauge"` + + ClientID string `tag:"client_id"` + Topic string `tag:"topic"` +} + +// writerStats is a struct that contains statistics on a writer. +// +// Since atomic is used to mutate the statistics the values must be 64-bit aligned. +// This is easily accomplished by always allocating this struct directly, (i.e. using a pointer to the struct). +// See https://golang.org/pkg/sync/atomic/#pkg-note-BUG +type writerStats struct { + dials counter + writes counter + messages counter + bytes counter + rebalances counter + errors counter + dialTime summary + writeTime summary + waitTime summary + retries summary + batchSize summary +} + +// NewWriter creates and returns a new Writer configured with config. +func NewWriter(config WriterConfig) *Writer { + if len(config.Brokers) == 0 { + panic("cannot create a kafka writer with an empty list of brokers") + } + + if len(config.Topic) == 0 { + panic("cannot create a kafka writer with an empty topic") + } + + if config.Dialer == nil { + config.Dialer = DefaultDialer + } + + if config.Balancer == nil { + config.Balancer = &RoundRobin{} + } + + if config.newPartitionWriter == nil { + config.newPartitionWriter = func(partition int, config WriterConfig, stats *writerStats) partitionWriter { + return newWriter(partition, config, stats) + } + } + + if config.MaxAttempts == 0 { + config.MaxAttempts = 10 + } + + if config.QueueCapacity == 0 { + config.QueueCapacity = 100 + } + + if config.BatchSize == 0 { + config.BatchSize = 100 + } + + if config.BatchTimeout == 0 { + config.BatchTimeout = 1 * time.Second + } + + if config.ReadTimeout == 0 { + config.ReadTimeout = 10 * time.Second + } + + if config.WriteTimeout == 0 { + config.WriteTimeout = 10 * time.Second + } + + if config.RebalanceInterval == 0 { + config.RebalanceInterval = 15 * time.Second + } + + w := &Writer{ + config: config, + msgs: make(chan writerMessage, config.QueueCapacity), + done: make(chan struct{}), + stats: &writerStats{ + dialTime: makeSummary(), + writeTime: makeSummary(), + waitTime: makeSummary(), + retries: makeSummary(), + }, + } + + w.join.Add(1) + go w.run() + return w +} + +// WriteMessages writes a batch of messages to the kafka topic configured on this +// writer. +// +// Unless the writer was configured to write messages asynchronously, the method +// blocks until all messages have been written, or until the maximum number of +// attempts was reached. +// +// When the method returns an error, there's no way to know yet which messages +// have succeeded of failed. +// +// The context passed as first argument may also be used to asynchronously +// cancel the operation. Note that in this case there are no guarantees made on +// whether messages were written to kafka. The program should assume that the +// whole batch failed and re-write the messages later (which could then cause +// duplicates). +func (w *Writer) WriteMessages(ctx context.Context, msgs ...Message) error { + if len(msgs) == 0 { + return nil + } + + var res = make(chan error, len(msgs)) + var err error + + t0 := time.Now() + + for attempt := 0; attempt < w.config.MaxAttempts; attempt++ { + w.mutex.RLock() + + if w.closed { + w.mutex.RUnlock() + return io.ErrClosedPipe + } + + for _, msg := range msgs { + select { + case w.msgs <- writerMessage{ + msg: msg, + res: res, + }: + case <-ctx.Done(): + w.mutex.RUnlock() + return ctx.Err() + } + } + + w.mutex.RUnlock() + + if w.config.Async { + break + } + + var retry []Message + + for i := 0; i != len(msgs); i++ { + select { + case e := <-res: + if e != nil { + if we, ok := e.(*writerError); ok { + w.stats.retries.observe(1) + retry, err = append(retry, we.msg), we.err + } else { + err = e + } + } + case <-ctx.Done(): + return ctx.Err() + } + } + + if msgs = retry; len(msgs) == 0 { + break + } + + timer := time.NewTimer(backoff(attempt+1, 100*time.Millisecond, 1*time.Second)) + select { + case <-timer.C: + // Only clear the error (so we retry the loop) if we have more retries, otherwise + // we risk silencing the error. + if attempt < w.config.MaxAttempts-1 { + err = nil + } + case <-ctx.Done(): + err = ctx.Err() + case <-w.done: + err = io.ErrClosedPipe + } + timer.Stop() + + if err != nil { + break + } + } + + t1 := time.Now() + w.stats.writeTime.observeDuration(t1.Sub(t0)) + + return err +} + +// Stats returns a snapshot of the writer stats since the last time the method +// was called, or since the writer was created if it is called for the first +// time. +// +// A typical use of this method is to spawn a goroutine that will periodically +// call Stats on a kafka writer and report the metrics to a stats collection +// system. +func (w *Writer) Stats() WriterStats { + return WriterStats{ + Dials: w.stats.dials.snapshot(), + Writes: w.stats.writes.snapshot(), + Messages: w.stats.messages.snapshot(), + Bytes: w.stats.bytes.snapshot(), + Rebalances: w.stats.rebalances.snapshot(), + Errors: w.stats.errors.snapshot(), + DialTime: w.stats.dialTime.snapshotDuration(), + WriteTime: w.stats.writeTime.snapshotDuration(), + WaitTime: w.stats.waitTime.snapshotDuration(), + Retries: w.stats.retries.snapshot(), + BatchSize: w.stats.batchSize.snapshot(), + MaxAttempts: int64(w.config.MaxAttempts), + MaxBatchSize: int64(w.config.BatchSize), + BatchTimeout: w.config.BatchTimeout, + ReadTimeout: w.config.ReadTimeout, + WriteTimeout: w.config.WriteTimeout, + RebalanceInterval: w.config.RebalanceInterval, + RequiredAcks: int64(w.config.RequiredAcks), + Async: w.config.Async, + QueueLength: int64(len(w.msgs)), + QueueCapacity: int64(cap(w.msgs)), + ClientID: w.config.Dialer.ClientID, + Topic: w.config.Topic, + } +} + +// Close flushes all buffered messages and closes the writer. The call to Close +// aborts any concurrent calls to WriteMessages, which then return with the +// io.ErrClosedPipe error. +func (w *Writer) Close() (err error) { + w.mutex.Lock() + + if !w.closed { + w.closed = true + close(w.msgs) + close(w.done) + } + + w.mutex.Unlock() + w.join.Wait() + return +} + +func (w *Writer) run() { + defer w.join.Done() + + ticker := time.NewTicker(w.config.RebalanceInterval) + defer ticker.Stop() + + var rebalance = true + var writers = make(map[int]partitionWriter) + var partitions []int + var err error + + for { + if rebalance { + w.stats.rebalances.observe(1) + rebalance = false + + var newPartitions []int + var oldPartitions = partitions + + if newPartitions, err = w.partitions(); err == nil { + for _, partition := range diffp(oldPartitions, newPartitions) { + w.close(writers[partition]) + delete(writers, partition) + } + + for _, partition := range diffp(newPartitions, oldPartitions) { + writers[partition] = w.open(partition) + } + + partitions = newPartitions + } + } + + select { + case wm, ok := <-w.msgs: + if !ok { + for _, writer := range writers { + w.close(writer) + } + return + } + + if len(partitions) != 0 { + selectedPartition := w.config.Balancer.Balance(wm.msg, partitions...) + writers[selectedPartition].messages() <- wm + } else { + // No partitions were found because the topic doesn't exist. + if err == nil { + err = fmt.Errorf("failed to find any partitions for topic %s", w.config.Topic) + } + + wm.res <- err + } + + case <-ticker.C: + rebalance = true + } + } +} + +func (w *Writer) partitions() (partitions []int, err error) { + for _, broker := range shuffledStrings(w.config.Brokers) { + var conn *Conn + var plist []Partition + + if conn, err = w.config.Dialer.Dial("tcp", broker); err != nil { + continue + } + + conn.SetReadDeadline(time.Now().Add(w.config.ReadTimeout)) + plist, err = conn.ReadPartitions(w.config.Topic) + conn.Close() + + if err == nil { + partitions = make([]int, len(plist)) + for i, p := range plist { + partitions[i] = p.ID + } + break + } + } + + sort.Ints(partitions) + return +} + +func (w *Writer) open(partition int) partitionWriter { + return w.config.newPartitionWriter(partition, w.config, w.stats) +} + +func (w *Writer) close(writer partitionWriter) { + w.join.Add(1) + go func() { + writer.close() + w.join.Done() + }() +} + +func diffp(new []int, old []int) (diff []int) { + for _, p := range new { + if i := sort.SearchInts(old, p); i == len(old) || old[i] != p { + diff = append(diff, p) + } + } + return +} + +type partitionWriter interface { + messages() chan<- writerMessage + close() +} + +type writer struct { + brokers []string + topic string + partition int + requiredAcks int + batchSize int + batchTimeout time.Duration + writeTimeout time.Duration + dialer *Dialer + msgs chan writerMessage + join sync.WaitGroup + stats *writerStats +} + +func newWriter(partition int, config WriterConfig, stats *writerStats) *writer { + w := &writer{ + brokers: config.Brokers, + topic: config.Topic, + partition: partition, + requiredAcks: config.RequiredAcks, + batchSize: config.BatchSize, + batchTimeout: config.BatchTimeout, + writeTimeout: config.WriteTimeout, + dialer: config.Dialer, + msgs: make(chan writerMessage, config.QueueCapacity), + stats: stats, + } + w.join.Add(1) + go w.run() + return w +} + +func (w *writer) close() { + close(w.msgs) + w.join.Wait() +} + +func (w *writer) messages() chan<- writerMessage { + return w.msgs +} + +func (w *writer) run() { + defer w.join.Done() + + ticker := time.NewTicker(w.batchTimeout / 10) + defer ticker.Stop() + + var conn *Conn + var done bool + var batch = make([]Message, 0, w.batchSize) + var resch = make([](chan<- error), 0, w.batchSize) + var lastFlushAt = time.Now() + + defer func() { + if conn != nil { + conn.Close() + } + }() + + for !done { + var mustFlush bool + + select { + case wm, ok := <-w.msgs: + if !ok { + done, mustFlush = true, true + } else { + batch = append(batch, wm.msg) + resch = append(resch, wm.res) + mustFlush = len(batch) >= w.batchSize + } + + case now := <-ticker.C: + mustFlush = now.Sub(lastFlushAt) > w.batchTimeout + } + + if mustFlush { + lastFlushAt = time.Now() + + if len(batch) == 0 { + continue + } + + var err error + if conn, err = w.write(conn, batch, resch); err != nil { + if conn != nil { + conn.Close() + conn = nil + } + } + + for i := range batch { + batch[i] = Message{} + } + + for i := range resch { + resch[i] = nil + } + + batch = batch[:0] + resch = resch[:0] + } + } +} + +func (w *writer) dial() (conn *Conn, err error) { + for _, broker := range shuffledStrings(w.brokers) { + t0 := time.Now() + if conn, err = w.dialer.DialLeader(context.Background(), "tcp", broker, w.topic, w.partition); err == nil { + t1 := time.Now() + w.stats.dials.observe(1) + w.stats.dialTime.observeDuration(t1.Sub(t0)) + conn.SetRequiredAcks(w.requiredAcks) + break + } + } + return +} + +func (w *writer) write(conn *Conn, batch []Message, resch [](chan<- error)) (ret *Conn, err error) { + w.stats.writes.observe(1) + if conn == nil { + if conn, err = w.dial(); err != nil { + w.stats.errors.observe(1) + for i, res := range resch { + res <- &writerError{msg: batch[i], err: err} + } + return + } + } + + t0 := time.Now() + conn.SetWriteDeadline(time.Now().Add(w.writeTimeout)) + + if _, err = conn.WriteMessages(batch...); err != nil { + w.stats.errors.observe(1) + for i, res := range resch { + res <- &writerError{msg: batch[i], err: err} + } + } else { + for _, m := range batch { + w.stats.messages.observe(1) + w.stats.bytes.observe(int64(len(m.Key) + len(m.Value))) + } + for _, res := range resch { + res <- nil + } + } + + t1 := time.Now() + w.stats.waitTime.observeDuration(t1.Sub(t0)) + w.stats.batchSize.observe(int64(len(batch))) + + ret = conn + return +} + +type writerMessage struct { + msg Message + res chan<- error +} + +type writerError struct { + msg Message + err error +} + +func (e *writerError) Cause() error { + return e.err +} + +func (e *writerError) Error() string { + return e.err.Error() +} + +func (e *writerError) Temporary() bool { + return isTemporary(e.err) +} + +func (e *writerError) Timeout() bool { + return isTimeout(e.err) +} + +func shuffledStrings(list []string) []string { + shuffledList := make([]string, len(list)) + copy(shuffledList, list) + + shufflerMutex.Lock() + + for i := range shuffledList { + j := shuffler.Intn(i + 1) + shuffledList[i], shuffledList[j] = shuffledList[j], shuffledList[i] + } + + shufflerMutex.Unlock() + return shuffledList +} + +var ( + shufflerMutex = sync.Mutex{} + shuffler = rand.New(rand.NewSource(time.Now().Unix())) +)